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

[1/7] nifi git commit: NIFI-3520 Refactoring instance class loading - Fixing FlowController to use appropriate class loader when instantiating processor - Updating ExtensionManager to leverage new flag in MANIFEST from NAR plugin - Adding ReloadComponent

Repository: nifi
Updated Branches:
  refs/heads/master 8f37ad451 -> 556f309df


http://git-wip-us.apache.org/repos/asf/nifi/blob/556f309d/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/FetchHDFS.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/FetchHDFS.java b/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/FetchHDFS.java
index 97039e2..08a8ce2 100644
--- a/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/FetchHDFS.java
+++ b/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/FetchHDFS.java
@@ -24,6 +24,7 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.compress.CompressionCodec;
 import org.apache.hadoop.io.compress.CompressionCodecFactory;
 import org.apache.hadoop.security.AccessControlException;
+import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.nifi.annotation.behavior.InputRequirement;
 import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
 import org.apache.nifi.annotation.behavior.Restricted;
@@ -46,6 +47,7 @@ import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.io.InputStream;
 import java.net.URI;
+import java.security.PrivilegedAction;
 import java.util.ArrayList;
 import java.util.HashSet;
 import java.util.List;
@@ -112,9 +114,10 @@ public class FetchHDFS extends AbstractHadoopProcessor {
         }
 
         final FileSystem hdfs = getFileSystem();
+        final UserGroupInformation ugi = getUserGroupInformation();
         final String filenameValue = context.getProperty(FILENAME).evaluateAttributeExpressions(flowFile).getValue();
 
-        Path path = null;
+        final Path path;
         try {
             path = new Path(filenameValue);
         } catch (IllegalArgumentException e) {
@@ -125,54 +128,64 @@ public class FetchHDFS extends AbstractHadoopProcessor {
             return;
         }
 
-        InputStream stream = null;
-        CompressionCodec codec = null;
-        Configuration conf = getConfiguration();
-        final CompressionCodecFactory compressionCodecFactory = new CompressionCodecFactory(conf);
-        final CompressionType compressionType = CompressionType.valueOf(context.getProperty(COMPRESSION_CODEC).toString());
-        final boolean inferCompressionCodec = compressionType == CompressionType.AUTOMATIC;
-
-        if(inferCompressionCodec) {
-            codec = compressionCodecFactory.getCodec(path);
-        } else if (compressionType != CompressionType.NONE) {
-            codec = getCompressionCodec(context, getConfiguration());
-        }
-
         final URI uri = path.toUri();
         final StopWatch stopWatch = new StopWatch(true);
-        try {
-
-            final String outputFilename;
-            final String originalFilename = path.getName();
-            stream = hdfs.open(path, 16384);
-
-            // Check if compression codec is defined (inferred or otherwise)
-            if (codec != null) {
-                stream = codec.createInputStream(stream);
-                outputFilename = StringUtils.removeEnd(originalFilename, codec.getDefaultExtension());
-            } else {
-                outputFilename = originalFilename;
+        final FlowFile finalFlowFile = flowFile;
+
+        ugi.doAs(new PrivilegedAction<Object>() {
+            @Override
+            public Object run() {
+                InputStream stream = null;
+                CompressionCodec codec = null;
+                Configuration conf = getConfiguration();
+                final CompressionCodecFactory compressionCodecFactory = new CompressionCodecFactory(conf);
+                final CompressionType compressionType = CompressionType.valueOf(context.getProperty(COMPRESSION_CODEC).toString());
+                final boolean inferCompressionCodec = compressionType == CompressionType.AUTOMATIC;
+
+                if(inferCompressionCodec) {
+                    codec = compressionCodecFactory.getCodec(path);
+                } else if (compressionType != CompressionType.NONE) {
+                    codec = getCompressionCodec(context, getConfiguration());
+                }
+
+                FlowFile flowFile = finalFlowFile;
+                try {
+                    final String outputFilename;
+                    final String originalFilename = path.getName();
+                    stream = hdfs.open(path, 16384);
+
+                    // Check if compression codec is defined (inferred or otherwise)
+                    if (codec != null) {
+                        stream = codec.createInputStream(stream);
+                        outputFilename = StringUtils.removeEnd(originalFilename, codec.getDefaultExtension());
+                    } else {
+                        outputFilename = originalFilename;
+                    }
+
+                    flowFile = session.importFrom(stream, finalFlowFile);
+                    flowFile = session.putAttribute(flowFile, CoreAttributes.FILENAME.key(), outputFilename);
+
+                    stopWatch.stop();
+                    getLogger().info("Successfully received content from {} for {} in {}", new Object[] {uri, flowFile, stopWatch.getDuration()});
+                    session.getProvenanceReporter().fetch(flowFile, uri.toString(), stopWatch.getDuration(TimeUnit.MILLISECONDS));
+                    session.transfer(flowFile, REL_SUCCESS);
+                } catch (final FileNotFoundException | AccessControlException e) {
+                    getLogger().error("Failed to retrieve content from {} for {} due to {}; routing to failure", new Object[] {uri, flowFile, e});
+                    flowFile = session.putAttribute(flowFile, "hdfs.failure.reason", e.getMessage());
+                    flowFile = session.penalize(flowFile);
+                    session.transfer(flowFile, REL_FAILURE);
+                } catch (final IOException e) {
+                    getLogger().error("Failed to retrieve content from {} for {} due to {}; routing to comms.failure", new Object[] {uri, flowFile, e});
+                    flowFile = session.penalize(flowFile);
+                    session.transfer(flowFile, REL_COMMS_FAILURE);
+                } finally {
+                    IOUtils.closeQuietly(stream);
+                }
+
+                return null;
             }
+        });
 
-            flowFile = session.importFrom(stream, flowFile);
-            flowFile = session.putAttribute(flowFile, CoreAttributes.FILENAME.key(), outputFilename);
-
-            stopWatch.stop();
-            getLogger().info("Successfully received content from {} for {} in {}", new Object[] {uri, flowFile, stopWatch.getDuration()});
-            session.getProvenanceReporter().fetch(flowFile, uri.toString(), stopWatch.getDuration(TimeUnit.MILLISECONDS));
-            session.transfer(flowFile, REL_SUCCESS);
-        } catch (final FileNotFoundException | AccessControlException e) {
-            getLogger().error("Failed to retrieve content from {} for {} due to {}; routing to failure", new Object[] {uri, flowFile, e});
-            flowFile = session.putAttribute(flowFile, "hdfs.failure.reason", e.getMessage());
-            flowFile = session.penalize(flowFile);
-            session.transfer(flowFile, REL_FAILURE);
-        } catch (final IOException e) {
-            getLogger().error("Failed to retrieve content from {} for {} due to {}; routing to comms.failure", new Object[] {uri, flowFile, e});
-            flowFile = session.penalize(flowFile);
-            session.transfer(flowFile, REL_COMMS_FAILURE);
-        } finally {
-            IOUtils.closeQuietly(stream);
-        }
     }
 
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/556f309d/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-reporting-tasks/src/test/java/org/apache/nifi/controller/MonitorMemoryTest.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-reporting-tasks/src/test/java/org/apache/nifi/controller/MonitorMemoryTest.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-reporting-tasks/src/test/java/org/apache/nifi/controller/MonitorMemoryTest.java
index aa49390..815b855 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-reporting-tasks/src/test/java/org/apache/nifi/controller/MonitorMemoryTest.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-reporting-tasks/src/test/java/org/apache/nifi/controller/MonitorMemoryTest.java
@@ -22,6 +22,7 @@ import org.apache.nifi.authorization.Authorizer;
 import org.apache.nifi.bundle.Bundle;
 import org.apache.nifi.controller.repository.FlowFileEventRepository;
 import org.apache.nifi.nar.ExtensionManager;
+import org.apache.nifi.nar.SystemBundle;
 import org.apache.nifi.provenance.MockProvenanceRepository;
 import org.apache.nifi.util.CapturingLogger;
 import org.apache.nifi.util.NiFiProperties;
@@ -151,7 +152,7 @@ public class MonitorMemoryTest {
         final NiFiProperties nifiProperties = NiFiProperties.createBasicNiFiProperties(null, addProps);
 
         // build the system bundle
-        final Bundle bundle = ExtensionManager.createSystemBundle(nifiProperties);
+        final Bundle bundle = SystemBundle.create(nifiProperties);
         ExtensionManager.discoverExtensions(bundle, Collections.emptySet());
 
         return new Tuple<>(FlowController.createStandaloneInstance(


[2/7] nifi git commit: NIFI-3520 Refactoring instance class loading - Fixing FlowController to use appropriate class loader when instantiating processor - Updating ExtensionManager to leverage new flag in MANIFEST from NAR plugin - Adding ReloadComponent

Posted by mc...@apache.org.
http://git-wip-us.apache.org/repos/asf/nifi/blob/556f309d/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-jetty/src/main/java/org/apache/nifi/web/server/JettyServer.java
----------------------------------------------------------------------
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 84c9671..8225892 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
@@ -26,7 +26,9 @@ import org.apache.nifi.bundle.BundleDetails;
 import org.apache.nifi.controller.UninheritableFlowException;
 import org.apache.nifi.controller.serialization.FlowSerializationException;
 import org.apache.nifi.controller.serialization.FlowSynchronizationException;
+import org.apache.nifi.documentation.DocGenerator;
 import org.apache.nifi.lifecycle.LifeCycleStartException;
+import org.apache.nifi.nar.ExtensionManager;
 import org.apache.nifi.nar.ExtensionMapping;
 import org.apache.nifi.security.util.KeyStoreUtils;
 import org.apache.nifi.services.FlowService;
@@ -113,7 +115,10 @@ public class JettyServer implements NiFiServer {
     private final Server server;
     private final NiFiProperties props;
 
+    private Bundle systemBundle;
+    private Set<Bundle> bundles;
     private ExtensionMapping extensionMapping;
+
     private WebAppContext webApiContext;
     private WebAppContext webDocsContext;
 
@@ -681,6 +686,11 @@ public class JettyServer implements NiFiServer {
     @Override
     public void start() {
         try {
+            ExtensionManager.discoverExtensions(systemBundle, bundles);
+            ExtensionManager.logClassLoaderMapping();
+
+            DocGenerator.generate(props, extensionMapping);
+
             // start the server
             server.start();
 
@@ -859,6 +869,12 @@ public class JettyServer implements NiFiServer {
     }
 
     @Override
+    public void setBundles(Bundle systemBundle, Set<Bundle> bundles) {
+        this.systemBundle = systemBundle;
+        this.bundles = bundles;
+    }
+
+    @Override
     public void stop() {
         try {
             server.stop();

http://git-wip-us.apache.org/repos/asf/nifi/blob/556f309d/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/authorization/AuthorizableLookup.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/authorization/AuthorizableLookup.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/authorization/AuthorizableLookup.java
index db5bd17..3b524bb 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/authorization/AuthorizableLookup.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/authorization/AuthorizableLookup.java
@@ -30,22 +30,23 @@ public interface AuthorizableLookup {
     Authorizable getController();
 
     /**
-     * Get the authorizable Processor.
+     * Get the authorizable for the given type and bundle. This will use a dummy instance of the
+     * component. The intent of this method is to provide access to the PropertyDescriptors
+     * prior to the component being created.
      *
-     * @param id processor id
+     * @param type component type
+     * @param bundle the bundle for the component
      * @return authorizable
      */
-    ConfigurableComponentAuthorizable getProcessor(String id);
+    ComponentAuthorizable getConfigurableComponent(String type, BundleDTO bundle);
 
     /**
-     * Get the authorizable for this Processor. This will create a dummy instance of the
-     * processor. The intent of this method is to provide access to the PropertyDescriptors
-     * prior to the component being created.
+     * Get the authorizable Processor.
      *
-     * @param type processor type
+     * @param id processor id
      * @return authorizable
      */
-    ConfigurableComponentAuthorizable getProcessorByType(String type, BundleDTO bundle);
+    ComponentAuthorizable getProcessor(String id);
 
     /**
      * Get the authorizable for querying Provenance.
@@ -139,18 +140,7 @@ public interface AuthorizableLookup {
      * @param id controller service id
      * @return authorizable
      */
-    ConfigurableComponentAuthorizable getControllerService(String id);
-
-    /**
-     * Get the authorizable for this Controller Service. This will create a dummy instance of the
-     * controller service. The intent of this method is to provide access to the PropertyDescriptors
-     * prior to the component being created.
-     *
-     * @param type controller service type
-     * @param bundle bundle
-     * @return authorizable
-     */
-    ConfigurableComponentAuthorizable getControllerServiceByType(String type, BundleDTO bundle);
+    ComponentAuthorizable getControllerService(String id);
 
     /**
      * Get the authorizable referencing component.
@@ -167,18 +157,7 @@ public interface AuthorizableLookup {
      * @param id reporting task id
      * @return authorizable
      */
-    ConfigurableComponentAuthorizable getReportingTask(String id);
-
-    /**
-     * Get the authorizable for this Reporting Task. This will create a dummy instance of the
-     * reporting task. The intent of this method is to provide access to the PropertyDescriptors
-     * prior to the component being created.
-     *
-     * @param type reporting task type
-     * @param bundle bundle
-     * @return authorizable
-     */
-    ConfigurableComponentAuthorizable getReportingTaskByType(String type, BundleDTO bundle);
+    ComponentAuthorizable getReportingTask(String id);
 
     /**
      * Get the authorizable Template.

http://git-wip-us.apache.org/repos/asf/nifi/blob/556f309d/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/authorization/AuthorizeControllerServiceReference.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/authorization/AuthorizeControllerServiceReference.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/authorization/AuthorizeControllerServiceReference.java
index e2ed926..16984fa 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/authorization/AuthorizeControllerServiceReference.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/authorization/AuthorizeControllerServiceReference.java
@@ -37,7 +37,7 @@ public final class AuthorizeControllerServiceReference {
      * @param authorizer authorizer
      * @param lookup lookup
      */
-    public static void authorizeControllerServiceReferences(final ConfigurableComponentAuthorizable authorizable, final Authorizer authorizer,
+    public static void authorizeControllerServiceReferences(final ComponentAuthorizable authorizable, final Authorizer authorizer,
                                                             final AuthorizableLookup lookup, final boolean authorizeTransitiveServices) {
 
         // consider each property when looking for service references
@@ -50,7 +50,7 @@ public final class AuthorizeControllerServiceReference {
                 // authorize the service if configured
                 if (serviceId != null) {
                     try {
-                        final ConfigurableComponentAuthorizable currentServiceAuthorizable = lookup.getControllerService(serviceId);
+                        final ComponentAuthorizable currentServiceAuthorizable = lookup.getControllerService(serviceId);
                         currentServiceAuthorizable.getAuthorizable().authorize(authorizer, RequestAction.READ, NiFiUserUtils.getNiFiUser());
 
                         if (authorizeTransitiveServices) {
@@ -72,7 +72,7 @@ public final class AuthorizeControllerServiceReference {
      * @param authorizer authorizer
      * @param lookup lookup
      */
-    public static void authorizeControllerServiceReferences(final Map<String, String> proposedProperties, final ConfigurableComponentAuthorizable authorizable,
+    public static void authorizeControllerServiceReferences(final Map<String, String> proposedProperties, final ComponentAuthorizable authorizable,
                                                             final Authorizer authorizer, final AuthorizableLookup lookup) {
 
         // only attempt to authorize if properties are changing

http://git-wip-us.apache.org/repos/asf/nifi/blob/556f309d/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/authorization/ComponentAuthorizable.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/authorization/ComponentAuthorizable.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/authorization/ComponentAuthorizable.java
new file mode 100644
index 0000000..1b7f8cd
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/authorization/ComponentAuthorizable.java
@@ -0,0 +1,69 @@
+/*
+ * 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.authorization;
+
+import org.apache.nifi.authorization.resource.Authorizable;
+import org.apache.nifi.components.PropertyDescriptor;
+
+import java.util.List;
+
+/**
+ * Authorizable for a component that references a ControllerService.
+ */
+public interface ComponentAuthorizable {
+    /**
+     * Returns the base authorizable for this ControllerServiceReference. Non null
+     *
+     * @return authorizable
+     */
+    Authorizable getAuthorizable();
+
+    /**
+     * Returns whether or not the underlying configurable component is restricted.
+     *
+     * @return whether or not the underlying configurable component is restricted
+     */
+    boolean isRestricted();
+
+    /**
+     * Returns the property descriptor for the specified property.
+     *
+     * @param propertyName property name
+     * @return property descriptor
+     */
+    PropertyDescriptor getPropertyDescriptor(String propertyName);
+
+    /**
+     * Returns the property descriptors for this configurable component.
+     *
+     * @return property descriptors
+     */
+    List<PropertyDescriptor> getPropertyDescriptors();
+
+    /**
+     * Returns the current value of the specified property.
+     *
+     * @param propertyDescriptor property descriptor
+     * @return value
+     */
+    String getValue(PropertyDescriptor propertyDescriptor);
+
+    /**
+     * Cleans up any resources resulting from the creation of these temporary components.
+     */
+    void cleanUpResources();
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/556f309d/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/authorization/ConfigurableComponentAuthorizable.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/authorization/ConfigurableComponentAuthorizable.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/authorization/ConfigurableComponentAuthorizable.java
deleted file mode 100644
index aaf1d3d..0000000
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/authorization/ConfigurableComponentAuthorizable.java
+++ /dev/null
@@ -1,69 +0,0 @@
-/*
- * 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.authorization;
-
-import org.apache.nifi.authorization.resource.Authorizable;
-import org.apache.nifi.components.PropertyDescriptor;
-
-import java.util.List;
-
-/**
- * Authorizable for a component that references a ControllerService.
- */
-public interface ConfigurableComponentAuthorizable {
-    /**
-     * Returns the base authorizable for this ControllerServiceReference. Non null
-     *
-     * @return authorizable
-     */
-    Authorizable getAuthorizable();
-
-    /**
-     * Returns whether or not the underlying configurable component is restricted.
-     *
-     * @return whether or not the underlying configurable component is restricted
-     */
-    boolean isRestricted();
-
-    /**
-     * Returns the property descriptor for the specified property.
-     *
-     * @param propertyName property name
-     * @return property descriptor
-     */
-    PropertyDescriptor getPropertyDescriptor(String propertyName);
-
-    /**
-     * Returns the property descriptors for this configurable component.
-     *
-     * @return property descriptors
-     */
-    List<PropertyDescriptor> getPropertyDescriptors();
-
-    /**
-     * Returns the current value of the specified property.
-     *
-     * @param propertyDescriptor property descriptor
-     * @return value
-     */
-    String getValue(PropertyDescriptor propertyDescriptor);
-
-    /**
-     * Cleans up any resources resulting from the creation of these temporary components.
-     */
-    void cleanUpResources();
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/556f309d/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/authorization/ProcessGroupAuthorizable.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/authorization/ProcessGroupAuthorizable.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/authorization/ProcessGroupAuthorizable.java
index 3e2fecc..f3516da 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/authorization/ProcessGroupAuthorizable.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/authorization/ProcessGroupAuthorizable.java
@@ -36,7 +36,7 @@ public interface ProcessGroupAuthorizable {
      *
      * @return all encapsulated processors
      */
-    Set<ConfigurableComponentAuthorizable> getEncapsulatedProcessors();
+    Set<ComponentAuthorizable> getEncapsulatedProcessors();
 
     /**
      * The authorizables for all encapsulated connections. Non null
@@ -99,6 +99,6 @@ public interface ProcessGroupAuthorizable {
      *
      * @return all encapsulated input ports
      */
-    Set<ConfigurableComponentAuthorizable> getEncapsulatedControllerServices();
+    Set<ComponentAuthorizable> getEncapsulatedControllerServices();
 
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/556f309d/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/authorization/SnippetAuthorizable.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/authorization/SnippetAuthorizable.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/authorization/SnippetAuthorizable.java
index e28bf74..7cc8e4c 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/authorization/SnippetAuthorizable.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/authorization/SnippetAuthorizable.java
@@ -36,7 +36,7 @@ public interface SnippetAuthorizable {
      *
      * @return processors
      */
-    Set<ConfigurableComponentAuthorizable> getSelectedProcessors();
+    Set<ComponentAuthorizable> getSelectedProcessors();
 
     /**
      * The authorizables for selected connections. Non null

http://git-wip-us.apache.org/repos/asf/nifi/blob/556f309d/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/authorization/StandardAuthorizableLookup.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/authorization/StandardAuthorizableLookup.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/authorization/StandardAuthorizableLookup.java
index ebc98de..5f8fb05 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/authorization/StandardAuthorizableLookup.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/authorization/StandardAuthorizableLookup.java
@@ -17,6 +17,7 @@
 package org.apache.nifi.authorization;
 
 import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.annotation.behavior.Restricted;
 import org.apache.nifi.authorization.resource.AccessPolicyAuthorizable;
 import org.apache.nifi.authorization.resource.Authorizable;
 import org.apache.nifi.authorization.resource.DataAuthorizable;
@@ -27,6 +28,7 @@ import org.apache.nifi.authorization.resource.RestrictedComponentsAuthorizable;
 import org.apache.nifi.authorization.resource.TenantAuthorizable;
 import org.apache.nifi.authorization.user.NiFiUser;
 import org.apache.nifi.bundle.BundleCoordinate;
+import org.apache.nifi.components.ConfigurableComponent;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.connectable.Connectable;
 import org.apache.nifi.connectable.Connection;
@@ -142,22 +144,22 @@ class StandardAuthorizableLookup implements AuthorizableLookup {
     }
 
     @Override
-    public ConfigurableComponentAuthorizable getProcessor(final String id) {
-        final ProcessorNode processorNode = processorDAO.getProcessor(id);
-        return new ProcessorConfigurableComponentAuthorizable(processorNode);
-    }
-
-    @Override
-    public ConfigurableComponentAuthorizable getProcessorByType(String type, BundleDTO bundle) {
+    public ComponentAuthorizable getConfigurableComponent(final String type, final BundleDTO bundle) {
         try {
-            final ProcessorNode processorNode = controllerFacade.createTemporaryProcessor(type, bundle);
-            return new ProcessorConfigurableComponentAuthorizable(processorNode);
+            final ConfigurableComponent configurableComponent = controllerFacade.getTemporaryComponent(type, bundle);
+            return new ConfigurableComponentAuthorizable(configurableComponent);
         } catch (final Exception e) {
-            throw new AccessDeniedException("Unable to create processor to verify if it references any Controller Services.");
+            throw new AccessDeniedException("Unable to create component to verify if it references any Controller Services.");
         }
     }
 
     @Override
+    public ComponentAuthorizable getProcessor(final String id) {
+        final ProcessorNode processorNode = processorDAO.getProcessor(id);
+        return new ProcessorComponentAuthorizable(processorNode);
+    }
+
+    @Override
     public RootGroupPortAuthorizable getRootGroupInputPort(String id) {
         final Port inputPort = inputPortDAO.getPort(id);
 
@@ -251,19 +253,9 @@ class StandardAuthorizableLookup implements AuthorizableLookup {
     }
 
     @Override
-    public ConfigurableComponentAuthorizable getControllerService(final String id) {
+    public ComponentAuthorizable getControllerService(final String id) {
         final ControllerServiceNode controllerService = controllerServiceDAO.getControllerService(id);
-        return new ControllerServiceConfigurableComponentAuthorizable(controllerService);
-    }
-
-    @Override
-    public ConfigurableComponentAuthorizable getControllerServiceByType(String type, BundleDTO bundle) {
-        try {
-            final ControllerServiceNode controllerService = controllerFacade.createTemporaryControllerService(type, bundle);
-            return new ControllerServiceConfigurableComponentAuthorizable(controllerService);
-        } catch (final Exception e) {
-            throw new AccessDeniedException("Unable to create controller service to verify if it references any Controller Services.");
-        }
+        return new ControllerServiceComponentAuthorizable(controllerService);
     }
 
     @Override
@@ -310,19 +302,9 @@ class StandardAuthorizableLookup implements AuthorizableLookup {
     }
 
     @Override
-    public ConfigurableComponentAuthorizable getReportingTask(final String id) {
+    public ComponentAuthorizable getReportingTask(final String id) {
         final ReportingTaskNode reportingTaskNode = reportingTaskDAO.getReportingTask(id);
-        return new ReportingTaskConfigurableComponentAuthorizable(reportingTaskNode);
-    }
-
-    @Override
-    public ConfigurableComponentAuthorizable getReportingTaskByType(String type, BundleDTO bundle) {
-        try {
-            final ReportingTaskNode reportingTask = controllerFacade.createTemporaryReportingTask(type, bundle);
-            return new ReportingTaskConfigurableComponentAuthorizable(reportingTask);
-        } catch (final Exception e) {
-            throw new AccessDeniedException("Unable to create reporting to verify if it references any Controller Services.");
-        }
+        return new ReportingTaskComponentAuthorizable(reportingTaskNode);
     }
 
     @Override
@@ -337,7 +319,7 @@ class StandardAuthorizableLookup implements AuthorizableLookup {
             }
 
             @Override
-            public Set<ConfigurableComponentAuthorizable> getSelectedProcessors() {
+            public Set<ComponentAuthorizable> getSelectedProcessors() {
                 return processGroup.getProcessors().stream()
                         .filter(processor -> snippet.getProcessors().containsKey(processor.getIdentifier()))
                         .map(processor -> getProcessor(processor.getIdentifier()))
@@ -616,8 +598,8 @@ class StandardAuthorizableLookup implements AuthorizableLookup {
      * @param controllerServices    controller services
      */
     private void createTemporaryProcessorsAndControllerServices(final FlowSnippetDTO snippet,
-                                                                final Set<ConfigurableComponentAuthorizable> processors,
-                                                                final Set<ConfigurableComponentAuthorizable> controllerServices) {
+                                                                final Set<ComponentAuthorizable> processors,
+                                                                final Set<ComponentAuthorizable> controllerServices) {
 
         if (snippet == null) {
             return;
@@ -627,7 +609,7 @@ class StandardAuthorizableLookup implements AuthorizableLookup {
             snippet.getProcessors().forEach(processor -> {
                 try {
                     final BundleCoordinate bundle = BundleUtils.getCompatibleBundle(processor.getType(), processor.getBundle());
-                    processors.add(getProcessorByType(processor.getType(), new BundleDTO(bundle.getGroup(), bundle.getId(), bundle.getVersion())));
+                    processors.add(getConfigurableComponent(processor.getType(), new BundleDTO(bundle.getGroup(), bundle.getId(), bundle.getVersion())));
                 } catch (final IllegalStateException e) {
                     // no compatible bundles... no additional auth checks necessary... if created, will be ghosted
                 }
@@ -638,7 +620,7 @@ class StandardAuthorizableLookup implements AuthorizableLookup {
             snippet.getControllerServices().forEach(controllerService -> {
                 try {
                     final BundleCoordinate bundle = BundleUtils.getCompatibleBundle(controllerService.getType(), controllerService.getBundle());
-                    controllerServices.add(getControllerServiceByType(controllerService.getType(), new BundleDTO(bundle.getGroup(), bundle.getId(), bundle.getVersion())));
+                    controllerServices.add(getConfigurableComponent(controllerService.getType(), new BundleDTO(bundle.getGroup(), bundle.getId(), bundle.getVersion())));
                 } catch (final IllegalStateException e) {
                     // no compatible bundles... no additional auth checks necessary... if created, will be ghosted
                 }
@@ -658,20 +640,20 @@ class StandardAuthorizableLookup implements AuthorizableLookup {
     @Override
     public TemplateContentsAuthorizable getTemplateContents(final FlowSnippetDTO snippet) {
         // templates are immutable so we can pre-compute all encapsulated processors and controller services
-        final Set<ConfigurableComponentAuthorizable> processors = new HashSet<>();
-        final Set<ConfigurableComponentAuthorizable> controllerServices = new HashSet<>();
+        final Set<ComponentAuthorizable> processors = new HashSet<>();
+        final Set<ComponentAuthorizable> controllerServices = new HashSet<>();
 
         // find all processors and controller services
         createTemporaryProcessorsAndControllerServices(snippet, processors, controllerServices);
 
         return new TemplateContentsAuthorizable() {
             @Override
-            public Set<ConfigurableComponentAuthorizable> getEncapsulatedProcessors() {
+            public Set<ComponentAuthorizable> getEncapsulatedProcessors() {
                 return processors;
             }
 
             @Override
-            public Set<ConfigurableComponentAuthorizable> getEncapsulatedControllerServices() {
+            public Set<ComponentAuthorizable> getEncapsulatedControllerServices() {
                 return controllerServices;
             }
         };
@@ -700,12 +682,54 @@ class StandardAuthorizableLookup implements AuthorizableLookup {
     }
 
     /**
-     * ConfigurableComponentAuthorizable for a ProcessorNode.
+     * ComponentAuthorizable for a ConfigurableComponent. This authorizable is intended only to be used when
+     * creating new components.
+     */
+    private static class ConfigurableComponentAuthorizable implements ComponentAuthorizable {
+        private final ConfigurableComponent configurableComponent;
+
+        public ConfigurableComponentAuthorizable(final ConfigurableComponent configurableComponent) {
+            this.configurableComponent = configurableComponent;
+        }
+
+        @Override
+        public Authorizable getAuthorizable() {
+            throw new UnsupportedOperationException();
+        }
+
+        @Override
+        public boolean isRestricted() {
+            return configurableComponent.getClass().isAnnotationPresent(Restricted.class);
+        }
+
+        @Override
+        public String getValue(PropertyDescriptor propertyDescriptor) {
+            return null;
+        }
+
+        @Override
+        public PropertyDescriptor getPropertyDescriptor(String propertyName) {
+            return configurableComponent.getPropertyDescriptor(propertyName);
+        }
+
+        @Override
+        public List<PropertyDescriptor> getPropertyDescriptors() {
+            return configurableComponent.getPropertyDescriptors();
+        }
+
+        @Override
+        public void cleanUpResources() {
+            ExtensionManager.removeInstanceClassLoader(configurableComponent.getIdentifier());
+        }
+    }
+
+    /**
+     * ComponentAuthorizable for a ProcessorNode.
      */
-    private static class ProcessorConfigurableComponentAuthorizable implements ConfigurableComponentAuthorizable {
+    private static class ProcessorComponentAuthorizable implements ComponentAuthorizable {
         private final ProcessorNode processorNode;
 
-        public ProcessorConfigurableComponentAuthorizable(ProcessorNode processorNode) {
+        public ProcessorComponentAuthorizable(ProcessorNode processorNode) {
             this.processorNode = processorNode;
         }
 
@@ -736,17 +760,17 @@ class StandardAuthorizableLookup implements AuthorizableLookup {
 
         @Override
         public void cleanUpResources() {
-            ExtensionManager.removeInstanceClassLoaderIfExists(processorNode.getIdentifier());
+            ExtensionManager.removeInstanceClassLoader(processorNode.getIdentifier());
         }
     }
 
     /**
-     * ConfigurableComponentAuthorizable for a ControllerServiceNode.
+     * ComponentAuthorizable for a ControllerServiceNode.
      */
-    private static class ControllerServiceConfigurableComponentAuthorizable implements ConfigurableComponentAuthorizable {
+    private static class ControllerServiceComponentAuthorizable implements ComponentAuthorizable {
         private final ControllerServiceNode controllerServiceNode;
 
-        public ControllerServiceConfigurableComponentAuthorizable(ControllerServiceNode controllerServiceNode) {
+        public ControllerServiceComponentAuthorizable(ControllerServiceNode controllerServiceNode) {
             this.controllerServiceNode = controllerServiceNode;
         }
 
@@ -777,17 +801,17 @@ class StandardAuthorizableLookup implements AuthorizableLookup {
 
         @Override
         public void cleanUpResources() {
-            ExtensionManager.removeInstanceClassLoaderIfExists(controllerServiceNode.getIdentifier());
+            ExtensionManager.removeInstanceClassLoader(controllerServiceNode.getIdentifier());
         }
     }
 
     /**
-     * ConfigurableComponentAuthorizable for a ProcessorNode.
+     * ComponentAuthorizable for a ProcessorNode.
      */
-    private static class ReportingTaskConfigurableComponentAuthorizable implements ConfigurableComponentAuthorizable {
+    private static class ReportingTaskComponentAuthorizable implements ComponentAuthorizable {
         private final ReportingTaskNode reportingTaskNode;
 
-        public ReportingTaskConfigurableComponentAuthorizable(ReportingTaskNode reportingTaskNode) {
+        public ReportingTaskComponentAuthorizable(ReportingTaskNode reportingTaskNode) {
             this.reportingTaskNode = reportingTaskNode;
         }
 
@@ -818,7 +842,7 @@ class StandardAuthorizableLookup implements AuthorizableLookup {
 
         @Override
         public void cleanUpResources() {
-            ExtensionManager.removeInstanceClassLoaderIfExists(reportingTaskNode.getIdentifier());
+            ExtensionManager.removeInstanceClassLoader(reportingTaskNode.getIdentifier());
         }
     }
 
@@ -835,9 +859,9 @@ class StandardAuthorizableLookup implements AuthorizableLookup {
         }
 
         @Override
-        public Set<ConfigurableComponentAuthorizable> getEncapsulatedProcessors() {
+        public Set<ComponentAuthorizable> getEncapsulatedProcessors() {
             return processGroup.findAllProcessors().stream().map(
-                    processorNode -> new ProcessorConfigurableComponentAuthorizable(processorNode)).collect(Collectors.toSet());
+                    processorNode -> new ProcessorComponentAuthorizable(processorNode)).collect(Collectors.toSet());
         }
 
         @Override
@@ -883,9 +907,9 @@ class StandardAuthorizableLookup implements AuthorizableLookup {
         }
 
         @Override
-        public Set<ConfigurableComponentAuthorizable> getEncapsulatedControllerServices() {
+        public Set<ComponentAuthorizable> getEncapsulatedControllerServices() {
             return processGroup.findAllControllerServices().stream().map(
-                    controllerServiceNode -> new ControllerServiceConfigurableComponentAuthorizable(controllerServiceNode)).collect(Collectors.toSet());
+                    controllerServiceNode -> new ControllerServiceComponentAuthorizable(controllerServiceNode)).collect(Collectors.toSet());
         }
     }
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/556f309d/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/authorization/TemplateContentsAuthorizable.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/authorization/TemplateContentsAuthorizable.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/authorization/TemplateContentsAuthorizable.java
index 7222765..8d7be9a 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/authorization/TemplateContentsAuthorizable.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/authorization/TemplateContentsAuthorizable.java
@@ -27,13 +27,13 @@ public interface TemplateContentsAuthorizable {
      *
      * @return temporary instances of all encapsulated processors
      */
-    Set<ConfigurableComponentAuthorizable> getEncapsulatedProcessors();
+    Set<ComponentAuthorizable> getEncapsulatedProcessors();
 
     /**
      * Returns temporary instances of all encapsulated controller services. Non null
      *
      * @return temporary instances of all encapsulated controller services
      */
-    Set<ConfigurableComponentAuthorizable> getEncapsulatedControllerServices();
+    Set<ComponentAuthorizable> getEncapsulatedControllerServices();
 
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/556f309d/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiServiceFacade.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiServiceFacade.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiServiceFacade.java
index 2ff0b3b..4179745 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiServiceFacade.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiServiceFacade.java
@@ -52,6 +52,7 @@ import org.apache.nifi.cluster.event.NodeEvent;
 import org.apache.nifi.cluster.manager.exception.IllegalNodeDeletionException;
 import org.apache.nifi.cluster.manager.exception.UnknownNodeException;
 import org.apache.nifi.cluster.protocol.NodeIdentifier;
+import org.apache.nifi.components.ConfigurableComponent;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.ValidationResult;
 import org.apache.nifi.components.Validator;
@@ -83,6 +84,7 @@ import org.apache.nifi.groups.RemoteProcessGroup;
 import org.apache.nifi.history.History;
 import org.apache.nifi.history.HistoryQuery;
 import org.apache.nifi.history.PreviousValue;
+import org.apache.nifi.processor.Processor;
 import org.apache.nifi.remote.RootGroupPort;
 import org.apache.nifi.reporting.Bulletin;
 import org.apache.nifi.reporting.BulletinQuery;
@@ -1678,8 +1680,8 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
                     }
 
                     try {
-                        final ControllerService controllerService = controllerFacade.createTemporaryControllerService(dto.getType(), dto.getBundle()).getControllerServiceImplementation();
-                        controllerService.getPropertyDescriptors().forEach(descriptor -> {
+                        final ConfigurableComponent configurableComponent = controllerFacade.getTemporaryComponent(dto.getType(), dto.getBundle());
+                        configurableComponent.getPropertyDescriptors().forEach(descriptor -> {
                             if (dto.getProperties().get(descriptor.getName()) == null) {
                                 dto.getProperties().put(descriptor.getName(), descriptor.getDefaultValue());
                             }
@@ -1702,8 +1704,8 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
                     }
 
                     try {
-                        final ProcessorNode processorNode = controllerFacade.createTemporaryProcessor(dto.getType(), dto.getBundle());
-                        processorNode.getPropertyDescriptors().forEach(descriptor -> {
+                        final ConfigurableComponent configurableComponent = controllerFacade.getTemporaryComponent(dto.getType(), dto.getBundle());
+                        configurableComponent.getPropertyDescriptors().forEach(descriptor -> {
                             if (config.getProperties().get(descriptor.getName()) == null) {
                                 config.getProperties().put(descriptor.getName(), descriptor.getDefaultValue());
                             }

http://git-wip-us.apache.org/repos/asf/nifi/blob/556f309d/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiWebConfigurationContext.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiWebConfigurationContext.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiWebConfigurationContext.java
index a0fae43..cbae778 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiWebConfigurationContext.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiWebConfigurationContext.java
@@ -45,7 +45,7 @@ import org.apache.nifi.authorization.AuthorizationResult;
 import org.apache.nifi.authorization.AuthorizationResult.Result;
 import org.apache.nifi.authorization.AuthorizeControllerServiceReference;
 import org.apache.nifi.authorization.Authorizer;
-import org.apache.nifi.authorization.ConfigurableComponentAuthorizable;
+import org.apache.nifi.authorization.ComponentAuthorizable;
 import org.apache.nifi.authorization.RequestAction;
 import org.apache.nifi.authorization.UserContextKeys;
 import org.apache.nifi.authorization.resource.Authorizable;
@@ -396,7 +396,7 @@ public class StandardNiFiWebConfigurationContext implements NiFiWebConfiguration
             // authorize access
             serviceFacade.authorizeAccess(lookup -> {
                 // authorize the processor
-                final ConfigurableComponentAuthorizable authorizable = lookup.getProcessor(id);
+                final ComponentAuthorizable authorizable = lookup.getProcessor(id);
                 authorizable.getAuthorizable().authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser());
 
                 // authorize any referenced service
@@ -587,7 +587,7 @@ public class StandardNiFiWebConfigurationContext implements NiFiWebConfiguration
             // authorize access
             serviceFacade.authorizeAccess(lookup -> {
                 // authorize the controller service
-                final ConfigurableComponentAuthorizable authorizable = lookup.getControllerService(id);
+                final ComponentAuthorizable authorizable = lookup.getControllerService(id);
                 authorizable.getAuthorizable().authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser());
 
                 // authorize any referenced service
@@ -752,7 +752,7 @@ public class StandardNiFiWebConfigurationContext implements NiFiWebConfiguration
             // authorize access
             serviceFacade.authorizeAccess(lookup -> {
                 // authorize the reporting task
-                final ConfigurableComponentAuthorizable authorizable = lookup.getReportingTask(id);
+                final ComponentAuthorizable authorizable = lookup.getReportingTask(id);
                 authorizable.getAuthorizable().authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser());
 
                 // authorize any referenced service

http://git-wip-us.apache.org/repos/asf/nifi/blob/556f309d/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ControllerResource.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ControllerResource.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ControllerResource.java
index 63f7c18..98400f2 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ControllerResource.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ControllerResource.java
@@ -30,7 +30,7 @@ import org.apache.nifi.authorization.AuthorizationResult;
 import org.apache.nifi.authorization.AuthorizationResult.Result;
 import org.apache.nifi.authorization.AuthorizeControllerServiceReference;
 import org.apache.nifi.authorization.Authorizer;
-import org.apache.nifi.authorization.ConfigurableComponentAuthorizable;
+import org.apache.nifi.authorization.ComponentAuthorizable;
 import org.apache.nifi.authorization.RequestAction;
 import org.apache.nifi.authorization.UserContextKeys;
 import org.apache.nifi.authorization.resource.ResourceFactory;
@@ -295,9 +295,9 @@ public class ControllerResource extends ApplicationResource {
                 lookup -> {
                     authorizeController(RequestAction.WRITE);
 
-                    ConfigurableComponentAuthorizable authorizable = null;
+                    ComponentAuthorizable authorizable = null;
                     try {
-                        authorizable = lookup.getReportingTaskByType(requestReportingTask.getType(), requestReportingTask.getBundle());
+                        authorizable = lookup.getConfigurableComponent(requestReportingTask.getType(), requestReportingTask.getBundle());
 
                         if (authorizable.isRestricted()) {
                             lookup.getRestrictedComponents().authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser());
@@ -401,9 +401,9 @@ public class ControllerResource extends ApplicationResource {
                 lookup -> {
                     authorizeController(RequestAction.WRITE);
 
-                    ConfigurableComponentAuthorizable authorizable = null;
+                    ComponentAuthorizable authorizable = null;
                     try {
-                        authorizable = lookup.getControllerServiceByType(requestControllerService.getType(), requestControllerService.getBundle());
+                        authorizable = lookup.getConfigurableComponent(requestControllerService.getType(), requestControllerService.getBundle());
 
                         if (authorizable.isRestricted()) {
                             lookup.getRestrictedComponents().authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser());

http://git-wip-us.apache.org/repos/asf/nifi/blob/556f309d/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ControllerServiceResource.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ControllerServiceResource.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ControllerServiceResource.java
index 46e95f1..e4ee044 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ControllerServiceResource.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ControllerServiceResource.java
@@ -25,7 +25,7 @@ import com.wordnik.swagger.annotations.Authorization;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.nifi.authorization.AuthorizeControllerServiceReference;
 import org.apache.nifi.authorization.Authorizer;
-import org.apache.nifi.authorization.ConfigurableComponentAuthorizable;
+import org.apache.nifi.authorization.ComponentAuthorizable;
 import org.apache.nifi.authorization.RequestAction;
 import org.apache.nifi.authorization.resource.Authorizable;
 import org.apache.nifi.authorization.user.NiFiUserUtils;
@@ -624,7 +624,7 @@ public class ControllerServiceResource extends ApplicationResource {
                 requestRevision,
                 lookup -> {
                     // authorize the service
-                    final ConfigurableComponentAuthorizable authorizable = lookup.getControllerService(id);
+                    final ComponentAuthorizable authorizable = lookup.getControllerService(id);
                     authorizable.getAuthorizable().authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser());
 
                     // authorize any referenced services
@@ -710,7 +710,7 @@ public class ControllerServiceResource extends ApplicationResource {
                 requestControllerServiceEntity,
                 requestRevision,
                 lookup -> {
-                    final ConfigurableComponentAuthorizable controllerService = lookup.getControllerService(id);
+                    final ComponentAuthorizable controllerService = lookup.getControllerService(id);
 
                     // ensure write permission to the controller service
                     controllerService.getAuthorizable().authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser());

http://git-wip-us.apache.org/repos/asf/nifi/blob/556f309d/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ProcessGroupResource.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ProcessGroupResource.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ProcessGroupResource.java
index ffcc084..1eb86fa 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ProcessGroupResource.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ProcessGroupResource.java
@@ -28,7 +28,7 @@ import org.apache.commons.lang3.StringUtils;
 import org.apache.nifi.authorization.AuthorizableLookup;
 import org.apache.nifi.authorization.AuthorizeControllerServiceReference;
 import org.apache.nifi.authorization.Authorizer;
-import org.apache.nifi.authorization.ConfigurableComponentAuthorizable;
+import org.apache.nifi.authorization.ComponentAuthorizable;
 import org.apache.nifi.authorization.ProcessGroupAuthorizable;
 import org.apache.nifi.authorization.RequestAction;
 import org.apache.nifi.authorization.SnippetAuthorizable;
@@ -657,9 +657,9 @@ public class ProcessGroupResource extends ApplicationResource {
                     final Authorizable processGroup = lookup.getProcessGroup(groupId).getAuthorizable();
                     processGroup.authorize(authorizer, RequestAction.WRITE, user);
 
-                    ConfigurableComponentAuthorizable authorizable = null;
+                    ComponentAuthorizable authorizable = null;
                     try {
-                        authorizable = lookup.getProcessorByType(requestProcessor.getType(), requestProcessor.getBundle());
+                        authorizable = lookup.getConfigurableComponent(requestProcessor.getType(), requestProcessor.getBundle());
 
                         if (authorizable.isRestricted()) {
                             lookup.getRestrictedComponents().authorize(authorizer, RequestAction.WRITE, user);
@@ -1806,7 +1806,7 @@ public class ProcessGroupResource extends ApplicationResource {
 
                     // flag to only perform the restricted check once, atomic reference so we can mark final and use in lambda
                     final AtomicBoolean restrictedCheckPerformed = new AtomicBoolean(false);
-                    final Consumer<ConfigurableComponentAuthorizable> authorizeRestricted = authorizable -> {
+                    final Consumer<ComponentAuthorizable> authorizeRestricted = authorizable -> {
                         if (authorizable.isRestricted() && restrictedCheckPerformed.compareAndSet(false, true)) {
                             lookup.getRestrictedComponents().authorize(authorizer, RequestAction.WRITE, user);
                         }
@@ -1982,7 +1982,7 @@ public class ProcessGroupResource extends ApplicationResource {
 
                     // flag to only perform the restricted check once, atomic reference so we can mark final and use in lambda
                     final AtomicBoolean restrictedCheckPerformed = new AtomicBoolean(false);
-                    final Consumer<ConfigurableComponentAuthorizable> authorizeRestricted = authorizable -> {
+                    final Consumer<ComponentAuthorizable> authorizeRestricted = authorizable -> {
                         if (authorizable.isRestricted() && restrictedCheckPerformed.compareAndSet(false, true)) {
                             lookup.getRestrictedComponents().authorize(authorizer, RequestAction.WRITE, user);
                         }
@@ -2347,9 +2347,9 @@ public class ProcessGroupResource extends ApplicationResource {
                     final Authorizable processGroup = lookup.getProcessGroup(groupId).getAuthorizable();
                     processGroup.authorize(authorizer, RequestAction.WRITE, user);
 
-                    ConfigurableComponentAuthorizable authorizable = null;
+                    ComponentAuthorizable authorizable = null;
                     try {
-                        authorizable = lookup.getControllerServiceByType(requestControllerService.getType(), requestControllerService.getBundle());
+                        authorizable = lookup.getConfigurableComponent(requestControllerService.getType(), requestControllerService.getBundle());
 
                         if (authorizable.isRestricted()) {
                             lookup.getRestrictedComponents().authorize(authorizer, RequestAction.WRITE, user);

http://git-wip-us.apache.org/repos/asf/nifi/blob/556f309d/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ProcessorResource.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ProcessorResource.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ProcessorResource.java
index 204d57b..125fe90 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ProcessorResource.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ProcessorResource.java
@@ -25,7 +25,7 @@ import com.wordnik.swagger.annotations.Authorization;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.nifi.authorization.AuthorizeControllerServiceReference;
 import org.apache.nifi.authorization.Authorizer;
-import org.apache.nifi.authorization.ConfigurableComponentAuthorizable;
+import org.apache.nifi.authorization.ComponentAuthorizable;
 import org.apache.nifi.authorization.RequestAction;
 import org.apache.nifi.authorization.resource.Authorizable;
 import org.apache.nifi.authorization.user.NiFiUser;
@@ -459,7 +459,7 @@ public class ProcessorResource extends ApplicationResource {
                 lookup -> {
                     final NiFiUser user = NiFiUserUtils.getNiFiUser();
 
-                    final ConfigurableComponentAuthorizable authorizable = lookup.getProcessor(id);
+                    final ComponentAuthorizable authorizable = lookup.getProcessor(id);
                     authorizable.getAuthorizable().authorize(authorizer, RequestAction.WRITE, user);
 
                     final ProcessorConfigDTO config = requestProcessorDTO.getConfig();
@@ -543,7 +543,7 @@ public class ProcessorResource extends ApplicationResource {
                 requestProcessorEntity,
                 requestRevision,
                 lookup -> {
-                    final ConfigurableComponentAuthorizable processor = lookup.getProcessor(id);
+                    final ComponentAuthorizable processor = lookup.getProcessor(id);
 
                     // ensure write permission to the processor
                     processor.getAuthorizable().authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser());

http://git-wip-us.apache.org/repos/asf/nifi/blob/556f309d/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ReportingTaskResource.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ReportingTaskResource.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ReportingTaskResource.java
index c62d24d..72b4431 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ReportingTaskResource.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ReportingTaskResource.java
@@ -25,7 +25,7 @@ import com.wordnik.swagger.annotations.Authorization;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.nifi.authorization.AuthorizeControllerServiceReference;
 import org.apache.nifi.authorization.Authorizer;
-import org.apache.nifi.authorization.ConfigurableComponentAuthorizable;
+import org.apache.nifi.authorization.ComponentAuthorizable;
 import org.apache.nifi.authorization.RequestAction;
 import org.apache.nifi.authorization.resource.Authorizable;
 import org.apache.nifi.authorization.user.NiFiUserUtils;
@@ -428,7 +428,7 @@ public class ReportingTaskResource extends ApplicationResource {
                 requestRevision,
                 lookup -> {
                     // authorize reporting task
-                    final ConfigurableComponentAuthorizable authorizable = lookup.getReportingTask(id);
+                    final ComponentAuthorizable authorizable = lookup.getReportingTask(id);
                     authorizable.getAuthorizable().authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser());
 
                     // authorize any referenced services
@@ -513,7 +513,7 @@ public class ReportingTaskResource extends ApplicationResource {
                 requestReportingTaskEntity,
                 requestRevision,
                 lookup -> {
-                    final ConfigurableComponentAuthorizable reportingTask = lookup.getReportingTask(id);
+                    final ComponentAuthorizable reportingTask = lookup.getReportingTask(id);
 
                     // ensure write permission to the reporting task
                     reportingTask.getAuthorizable().authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser());

http://git-wip-us.apache.org/repos/asf/nifi/blob/556f309d/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerFacade.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerFacade.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerFacade.java
index d31d692..88ce4e0 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerFacade.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerFacade.java
@@ -32,6 +32,7 @@ import org.apache.nifi.authorization.user.NiFiUserUtils;
 import org.apache.nifi.bundle.Bundle;
 import org.apache.nifi.bundle.BundleCoordinate;
 import org.apache.nifi.cluster.protocol.NodeIdentifier;
+import org.apache.nifi.components.ConfigurableComponent;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.connectable.Connectable;
 import org.apache.nifi.connectable.Connection;
@@ -45,11 +46,9 @@ import org.apache.nifi.controller.ProcessorNode;
 import org.apache.nifi.controller.ReportingTaskNode;
 import org.apache.nifi.controller.ScheduledState;
 import org.apache.nifi.controller.Template;
-import org.apache.nifi.controller.exception.ProcessorInstantiationException;
 import org.apache.nifi.controller.label.Label;
 import org.apache.nifi.controller.queue.FlowFileQueue;
 import org.apache.nifi.controller.queue.QueueSize;
-import org.apache.nifi.controller.reporting.ReportingTaskInstantiationException;
 import org.apache.nifi.controller.repository.ContentNotFoundException;
 import org.apache.nifi.controller.repository.claim.ContentDirection;
 import org.apache.nifi.controller.service.ControllerServiceNode;
@@ -134,7 +133,6 @@ import java.util.Set;
 import java.util.SortedSet;
 import java.util.TimeZone;
 import java.util.TreeSet;
-import java.util.UUID;
 import java.util.concurrent.TimeUnit;
 import java.util.function.Consumer;
 import java.util.stream.Collectors;
@@ -200,35 +198,21 @@ public class ControllerFacade implements Authorizable {
     }
 
     /**
-     * Create a temporary Processor used for extracting PropertyDescriptor's for ControllerService reference authorization.
+     * Gets the cached temporary instance of the component for the given type and bundle.
      *
-     * @param type type of processor
-     * @return processor
-     * @throws ProcessorInstantiationException when unable to instantiate the processor
+     * @param type type of the component
+     * @param bundle the bundle of the component
+     * @return the temporary component
+     * @throws IllegalStateException if no temporary component exists for the given type and bundle
      */
-    public ProcessorNode createTemporaryProcessor(String type, BundleDTO bundle) throws ProcessorInstantiationException {
-        return flowController.createProcessor(type, UUID.randomUUID().toString(), BundleUtils.getBundle(type, bundle), false);
-    }
+    public ConfigurableComponent getTemporaryComponent(final String type, final BundleDTO bundle) {
+        final ConfigurableComponent configurableComponent = ExtensionManager.getTempComponent(type, BundleUtils.getBundle(type, bundle));
 
-    /**
-     * Create a temporary ReportingTask used for extracting PropertyDescriptor's for ControllerService reference authorization.
-     *
-     * @param type type of reporting task
-     * @return reporting task
-     * @throws ReportingTaskInstantiationException when unable to instantiate the reporting task
-     */
-    public ReportingTaskNode createTemporaryReportingTask(String type, BundleDTO bundle) throws ReportingTaskInstantiationException {
-        return flowController.createReportingTask(type, UUID.randomUUID().toString(), BundleUtils.getBundle(type, bundle), false, false);
-    }
+        if (configurableComponent == null) {
+            throw new IllegalStateException("Unable to obtain temporary component for " + type);
+        }
 
-    /**
-     * Create a temporary ControllerService used for extracting PropertyDescriptor's for ControllerService reference authorization.
-     *
-     * @param type type of controller service
-     * @return controller service
-     */
-    public ControllerServiceNode createTemporaryControllerService(String type, BundleDTO bundle) {
-        return flowController.createControllerService(type, UUID.randomUUID().toString(), BundleUtils.getBundle(type, bundle), false);
+        return configurableComponent;
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/nifi/blob/556f309d/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardControllerServiceDAO.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardControllerServiceDAO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardControllerServiceDAO.java
index c563e49..1649370 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardControllerServiceDAO.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardControllerServiceDAO.java
@@ -77,7 +77,8 @@ public class StandardControllerServiceDAO extends ComponentDAO implements Contro
         try {
             // create the controller service
             final ControllerServiceNode controllerService = serviceProvider.createControllerService(
-                    controllerServiceDTO.getType(), controllerServiceDTO.getId(), BundleUtils.getBundle(controllerServiceDTO.getType(), controllerServiceDTO.getBundle()), true);
+                    controllerServiceDTO.getType(), controllerServiceDTO.getId(), BundleUtils.getBundle(controllerServiceDTO.getType(),
+                            controllerServiceDTO.getBundle()), Collections.emptySet(), true);
 
             // ensure we can perform the update
             verifyUpdate(controllerService, controllerServiceDTO);
@@ -170,7 +171,7 @@ public class StandardControllerServiceDAO extends ComponentDAO implements Contro
         if (bundleDTO != null) {
             final BundleCoordinate incomingCoordinate = BundleUtils.getBundle(controllerService.getCanonicalClassName(), bundleDTO);
             try {
-                flowController.changeControllerServiceType(controllerService, controllerService.getCanonicalClassName(), incomingCoordinate);
+                flowController.reload(controllerService, controllerService.getCanonicalClassName(), incomingCoordinate, Collections.emptySet());
             } catch (ControllerServiceInstantiationException e) {
                 throw new NiFiCoreException(String.format("Unable to update controller service %s from %s to %s due to: %s",
                         controllerServiceDTO.getId(), controllerService.getBundleCoordinate().getCoordinate(), incomingCoordinate.getCoordinate(), e.getMessage()), e);

http://git-wip-us.apache.org/repos/asf/nifi/blob/556f309d/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardProcessorDAO.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardProcessorDAO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardProcessorDAO.java
index 9b9ab4e..ddc7b98 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardProcessorDAO.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardProcessorDAO.java
@@ -48,6 +48,7 @@ import org.slf4j.LoggerFactory;
 
 import java.text.ParseException;
 import java.util.ArrayList;
+import java.util.Collections;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
@@ -453,7 +454,7 @@ public class StandardProcessorDAO extends ComponentDAO implements ProcessorDAO {
         if (bundleDTO != null) {
             BundleCoordinate incomingCoordinate = BundleUtils.getBundle(processor.getCanonicalClassName(), bundleDTO);
             try {
-                flowController.changeProcessorType(processor, processor.getCanonicalClassName(), incomingCoordinate);
+                flowController.reload(processor, processor.getCanonicalClassName(), incomingCoordinate, Collections.emptySet());
             } catch (ProcessorInstantiationException e) {
                 throw new NiFiCoreException(String.format("Unable to update processor %s from %s to %s due to: %s",
                         processorDTO.getId(), processor.getBundleCoordinate().getCoordinate(), incomingCoordinate.getCoordinate(), e.getMessage()), e);

http://git-wip-us.apache.org/repos/asf/nifi/blob/556f309d/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardReportingTaskDAO.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardReportingTaskDAO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardReportingTaskDAO.java
index 7a4df89..3d406f5 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardReportingTaskDAO.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardReportingTaskDAO.java
@@ -20,6 +20,7 @@ import org.apache.commons.lang3.StringUtils;
 import org.apache.nifi.bundle.BundleCoordinate;
 import org.apache.nifi.components.state.Scope;
 import org.apache.nifi.components.state.StateMap;
+import org.apache.nifi.controller.ReloadComponent;
 import org.apache.nifi.controller.ReportingTaskNode;
 import org.apache.nifi.controller.ScheduledState;
 import org.apache.nifi.controller.exception.ComponentLifeCycleException;
@@ -39,6 +40,7 @@ import org.quartz.CronExpression;
 
 import java.text.ParseException;
 import java.util.ArrayList;
+import java.util.Collections;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
@@ -49,6 +51,7 @@ public class StandardReportingTaskDAO extends ComponentDAO implements ReportingT
 
     private ReportingTaskProvider reportingTaskProvider;
     private ComponentStateDAO componentStateDAO;
+    private ReloadComponent reloadComponent;
 
     private ReportingTaskNode locateReportingTask(final String reportingTaskId) {
         // get the reporting task
@@ -167,7 +170,7 @@ public class StandardReportingTaskDAO extends ComponentDAO implements ReportingT
         if (bundleDTO != null) {
             final BundleCoordinate incomingCoordinate = BundleUtils.getBundle(reportingTask.getCanonicalClassName(), bundleDTO);
             try {
-                reportingTaskProvider.changeReportingTaskType(reportingTask, reportingTask.getCanonicalClassName(), incomingCoordinate);
+                reloadComponent.reload(reportingTask, reportingTask.getCanonicalClassName(), incomingCoordinate, Collections.emptySet());
             } catch (ReportingTaskInstantiationException e) {
                 throw new NiFiCoreException(String.format("Unable to update reporting task %s from %s to %s due to: %s",
                         reportingTaskDTO.getId(), reportingTask.getBundleCoordinate().getCoordinate(), incomingCoordinate.getCoordinate(), e.getMessage()), e);
@@ -356,4 +359,8 @@ public class StandardReportingTaskDAO extends ComponentDAO implements ReportingT
     public void setComponentStateDAO(ComponentStateDAO componentStateDAO) {
         this.componentStateDAO = componentStateDAO;
     }
+
+    public void setReloadComponent(ReloadComponent reloadComponent) {
+        this.reloadComponent = reloadComponent;
+    }
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/556f309d/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/resources/nifi-web-api-context.xml
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/resources/nifi-web-api-context.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/resources/nifi-web-api-context.xml
index bdc042a..ca8a7e1 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/resources/nifi-web-api-context.xml
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/resources/nifi-web-api-context.xml
@@ -97,6 +97,7 @@
     <bean id="reportingTaskDAO" class="org.apache.nifi.web.dao.impl.StandardReportingTaskDAO">
         <property name="reportingTaskProvider" ref="reportingTaskProvider"/>
         <property name="componentStateDAO" ref="componentStateDAO"/>
+        <property name="reloadComponent" ref="flowController" />
     </bean>
     <bean id="componentStateDAO" class="org.apache.nifi.web.dao.impl.StandardComponentStateDAO">
         <property name="stateManagerProvider" ref="stateManagerProvider"/>

http://git-wip-us.apache.org/repos/asf/nifi/blob/556f309d/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/integration/accesscontrol/AccessControlHelper.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/integration/accesscontrol/AccessControlHelper.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/integration/accesscontrol/AccessControlHelper.java
index 6476ed8..b425856 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/integration/accesscontrol/AccessControlHelper.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/integration/accesscontrol/AccessControlHelper.java
@@ -24,6 +24,7 @@ import org.apache.nifi.integration.util.NiFiTestServer;
 import org.apache.nifi.integration.util.NiFiTestUser;
 import org.apache.nifi.nar.ExtensionManager;
 import org.apache.nifi.nar.NarClassLoaders;
+import org.apache.nifi.nar.SystemBundle;
 import org.apache.nifi.util.NiFiProperties;
 
 import java.io.File;
@@ -64,7 +65,7 @@ public class AccessControlHelper {
         flowXmlPath = props.getProperty(NiFiProperties.FLOW_CONFIGURATION_FILE);
 
         // load extensions
-        final Bundle systemBundle = ExtensionManager.createSystemBundle(props);
+        final Bundle systemBundle = SystemBundle.create(props);
         NarClassLoaders.getInstance().init(props.getFrameworkWorkingDirectory(), props.getExtensionsWorkingDirectory());
         ExtensionManager.discoverExtensions(systemBundle, NarClassLoaders.getInstance().getBundles());
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/556f309d/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/integration/accesscontrol/ITAccessTokenEndpoint.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/integration/accesscontrol/ITAccessTokenEndpoint.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/integration/accesscontrol/ITAccessTokenEndpoint.java
index 2d345e1..8deffe7 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/integration/accesscontrol/ITAccessTokenEndpoint.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/integration/accesscontrol/ITAccessTokenEndpoint.java
@@ -25,6 +25,7 @@ import org.apache.nifi.integration.util.NiFiTestUser;
 import org.apache.nifi.integration.util.SourceTestProcessor;
 import org.apache.nifi.nar.ExtensionManager;
 import org.apache.nifi.nar.NarClassLoaders;
+import org.apache.nifi.nar.SystemBundle;
 import org.apache.nifi.security.util.SslContextFactory;
 import org.apache.nifi.util.NiFiProperties;
 import org.apache.nifi.web.api.dto.AccessConfigurationDTO;
@@ -71,7 +72,7 @@ public class ITAccessTokenEndpoint {
         FileUtils.deleteDirectory(props.getDatabaseRepositoryPath().toFile());
 
         // load extensions
-        final Bundle systemBundle = ExtensionManager.createSystemBundle(props);
+        final Bundle systemBundle = SystemBundle.create(props);
         NarClassLoaders.getInstance().init(props.getFrameworkWorkingDirectory(), props.getExtensionsWorkingDirectory());
         ExtensionManager.discoverExtensions(systemBundle, NarClassLoaders.getInstance().getBundles());
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/556f309d/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/web/StandardNiFiServiceFacadeTest.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/web/StandardNiFiServiceFacadeTest.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/web/StandardNiFiServiceFacadeTest.java
index 373c318..9933d4b 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/web/StandardNiFiServiceFacadeTest.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/web/StandardNiFiServiceFacadeTest.java
@@ -25,7 +25,7 @@ import org.apache.nifi.authorization.AuthorizableLookup;
 import org.apache.nifi.authorization.AuthorizationRequest;
 import org.apache.nifi.authorization.AuthorizationResult;
 import org.apache.nifi.authorization.Authorizer;
-import org.apache.nifi.authorization.ConfigurableComponentAuthorizable;
+import org.apache.nifi.authorization.ComponentAuthorizable;
 import org.apache.nifi.authorization.Resource;
 import org.apache.nifi.authorization.resource.Authorizable;
 import org.apache.nifi.authorization.resource.ResourceFactory;
@@ -114,7 +114,7 @@ public class StandardNiFiServiceFacadeTest {
             }
 
             // component authorizable
-            final ConfigurableComponentAuthorizable componentAuthorizable = mock(ConfigurableComponentAuthorizable.class);
+            final ComponentAuthorizable componentAuthorizable = mock(ComponentAuthorizable.class);
             when(componentAuthorizable.getAuthorizable()).then(getAuthorizableInvocation -> {
 
                 // authorizable

http://git-wip-us.apache.org/repos/asf/nifi/blob/556f309d/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/pom.xml
----------------------------------------------------------------------
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 f803a36..45494d4 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/pom.xml
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/pom.xml
@@ -32,6 +32,7 @@
         <module>nifi-framework-core</module>
         <module>nifi-framework-cluster-protocol</module>
         <module>nifi-framework-cluster</module>
+        <module>nifi-framework-nar-utils</module>
         <module>nifi-user-actions</module>
         <module>nifi-framework-authorization</module>
         <module>nifi-file-authorizer</module>

http://git-wip-us.apache.org/repos/asf/nifi/blob/556f309d/nifi-nar-bundles/nifi-framework-bundle/pom.xml
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/pom.xml b/nifi-nar-bundles/nifi-framework-bundle/pom.xml
index a42a093..1ea79c0 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/pom.xml
+++ b/nifi-nar-bundles/nifi-framework-bundle/pom.xml
@@ -65,6 +65,11 @@
             </dependency>
             <dependency>
                 <groupId>org.apache.nifi</groupId>
+                <artifactId>nifi-framework-nar-utils</artifactId>
+                <version>1.2.0-SNAPSHOT</version>
+            </dependency>
+            <dependency>
+                <groupId>org.apache.nifi</groupId>
                 <artifactId>nifi-site-to-site</artifactId>
                 <version>1.2.0-SNAPSHOT</version>
             </dependency>

http://git-wip-us.apache.org/repos/asf/nifi/blob/556f309d/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hadoop-nar/pom.xml
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hadoop-nar/pom.xml b/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hadoop-nar/pom.xml
index 5e2d3d3..c307128 100644
--- a/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hadoop-nar/pom.xml
+++ b/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hadoop-nar/pom.xml
@@ -28,7 +28,7 @@
     <dependencies>
         <dependency>
             <groupId>org.apache.nifi</groupId>
-            <artifactId>nifi-standard-services-api-nar</artifactId>
+            <artifactId>nifi-hadoop-libraries-nar</artifactId>
             <type>nar</type>
         </dependency>
         <dependency>

http://git-wip-us.apache.org/repos/asf/nifi/blob/556f309d/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/pom.xml
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/pom.xml b/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/pom.xml
index 3002bcc..7429ea8 100644
--- a/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/pom.xml
+++ b/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/pom.xml
@@ -44,15 +44,21 @@
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-flowfile-packager</artifactId>
         </dependency>
-        <dependency> 
-            <groupId>org.apache.hadoop</groupId> 
-            <artifactId>hadoop-client</artifactId>
+        <dependency>
+            <groupId>org.apache.hadoop</groupId>
+            <artifactId>hadoop-common</artifactId>
+            <scope>provided</scope>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-distributed-cache-client-service-api</artifactId>
         </dependency>
         <dependency>
+            <groupId>org.apache.hadoop</groupId>
+            <artifactId>hadoop-hdfs</artifactId>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-mock</artifactId>
             <scope>test</scope>

http://git-wip-us.apache.org/repos/asf/nifi/blob/556f309d/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/AbstractHadoopProcessor.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/AbstractHadoopProcessor.java b/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/AbstractHadoopProcessor.java
index 7a8c34c..dac2b30 100644
--- a/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/AbstractHadoopProcessor.java
+++ b/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/AbstractHadoopProcessor.java
@@ -37,6 +37,7 @@ import org.apache.nifi.components.ValidationResult;
 import org.apache.nifi.components.Validator;
 import org.apache.nifi.hadoop.KerberosProperties;
 import org.apache.nifi.hadoop.SecurityUtil;
+import org.apache.nifi.logging.ComponentLog;
 import org.apache.nifi.processor.AbstractProcessor;
 import org.apache.nifi.processor.ProcessContext;
 import org.apache.nifi.processor.ProcessorInitializationContext;
@@ -45,7 +46,6 @@ import org.apache.nifi.processor.util.StandardValidators;
 import org.apache.nifi.util.StringUtils;
 
 import javax.net.SocketFactory;
-
 import java.io.File;
 import java.io.IOException;
 import java.lang.ref.WeakReference;
@@ -65,7 +65,7 @@ import java.util.concurrent.atomic.AtomicReference;
 /**
  * This is a base class that is helpful when building processors interacting with HDFS.
  */
-@RequiresInstanceClassLoading
+@RequiresInstanceClassLoading(cloneAncestorResources = true)
 public abstract class AbstractHadoopProcessor extends AbstractProcessor {
     /**
      * Compression Type Enum
@@ -191,7 +191,9 @@ public abstract class AbstractHadoopProcessor extends AbstractProcessor {
                 // then load the Configuration and set the new resources in the holder
                 if (resources == null || !configResources.equals(resources.getConfigResources())) {
                     getLogger().debug("Reloading validation resources");
-                    resources = new ValidationResources(configResources, getConfigurationFromResources(configResources));
+                    final Configuration config = new ExtendedConfiguration(getLogger());
+                    config.setClassLoader(Thread.currentThread().getContextClassLoader());
+                    resources = new ValidationResources(configResources, getConfigurationFromResources(config, configResources));
                     validationResourceHolder.set(resources);
                 }
 
@@ -240,9 +242,8 @@ public abstract class AbstractHadoopProcessor extends AbstractProcessor {
         hdfsResources.set(new HdfsResources(null, null, null));
     }
 
-    private static Configuration getConfigurationFromResources(String configResources) throws IOException {
+    private static Configuration getConfigurationFromResources(final Configuration config, String configResources) throws IOException {
         boolean foundResources = false;
-        final Configuration config = new ExtendedConfiguration();
         if (null != configResources) {
             String[] resources = configResources.split(",");
             for (String resource : resources) {
@@ -272,8 +273,10 @@ public abstract class AbstractHadoopProcessor extends AbstractProcessor {
      * Reset Hadoop Configuration and FileSystem based on the supplied configuration resources.
      */
     HdfsResources resetHDFSResources(String configResources, ProcessContext context) throws IOException {
-        Configuration config = getConfigurationFromResources(configResources);
-        config.setClassLoader(Thread.currentThread().getContextClassLoader()); // set the InstanceClassLoader
+        Configuration config = new ExtendedConfiguration(getLogger());
+        config.setClassLoader(Thread.currentThread().getContextClassLoader());
+
+        getConfigurationFromResources(config, configResources);
 
         // first check for timeout on HDFS connection, because FileSystem has a hard coded 15 minute timeout
         checkHdfsUriForTimeout(config);
@@ -531,16 +534,22 @@ public abstract class AbstractHadoopProcessor extends AbstractProcessor {
      */
     static class ExtendedConfiguration extends Configuration {
 
+        private final ComponentLog logger;
         private final Map<ClassLoader, Map<String, WeakReference<Class<?>>>> CACHE_CLASSES = new WeakHashMap<>();
 
+        public ExtendedConfiguration(final ComponentLog logger) {
+            this.logger = logger;
+        }
+
         public Class<?> getClassByNameOrNull(String name) {
-            Map<String, WeakReference<Class<?>>> map;
+            final ClassLoader classLoader = getClassLoader();
 
+            Map<String, WeakReference<Class<?>>> map;
             synchronized (CACHE_CLASSES) {
-                map = CACHE_CLASSES.get(getClassLoader());
+                map = CACHE_CLASSES.get(classLoader);
                 if (map == null) {
                     map = Collections.synchronizedMap(new WeakHashMap<>());
-                    CACHE_CLASSES.put(getClassLoader(), map);
+                    CACHE_CLASSES.put(classLoader, map);
                 }
             }
 
@@ -552,9 +561,9 @@ public abstract class AbstractHadoopProcessor extends AbstractProcessor {
 
             if (clazz == null) {
                 try {
-                    clazz = Class.forName(name, true, getClassLoader());
+                    clazz = Class.forName(name, true, classLoader);
                 } catch (ClassNotFoundException e) {
-                    e.printStackTrace();
+                    logger.error(e.getMessage(), e);
                     return null;
                 }
                 // two putters can race here, but they'll put the same class


[5/7] nifi git commit: NIFI-3520 Refactoring instance class loading - Fixing FlowController to use appropriate class loader when instantiating processor - Updating ExtensionManager to leverage new flag in MANIFEST from NAR plugin - Adding ReloadComponent

Posted by mc...@apache.org.
http://git-wip-us.apache.org/repos/asf/nifi/blob/556f309d/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/init/ConfigurableComponentInitializerFactory.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/init/ConfigurableComponentInitializerFactory.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/init/ConfigurableComponentInitializerFactory.java
new file mode 100644
index 0000000..f6ab922
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/init/ConfigurableComponentInitializerFactory.java
@@ -0,0 +1,44 @@
+/*
+ * 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.init;
+
+import org.apache.nifi.components.ConfigurableComponent;
+import org.apache.nifi.controller.ControllerService;
+import org.apache.nifi.processor.Processor;
+import org.apache.nifi.reporting.ReportingTask;
+
+public class ConfigurableComponentInitializerFactory {
+
+    /**
+     * Returns a ConfigurableComponentInitializer for the type of component.
+     * Currently Processor, ControllerService and ReportingTask are supported.
+     *
+     * @param componentClass the class that requires a ConfigurableComponentInitializer
+     * @return a ConfigurableComponentInitializer capable of initializing that specific type of class
+     */
+    public static ConfigurableComponentInitializer createComponentInitializer(final Class<? extends ConfigurableComponent> componentClass) {
+        if (Processor.class.isAssignableFrom(componentClass)) {
+            return new ProcessorInitializer();
+        } else if (ControllerService.class.isAssignableFrom(componentClass)) {
+            return new ControllerServiceInitializer();
+        } else if (ReportingTask.class.isAssignableFrom(componentClass)) {
+            return new ReportingTaskingInitializer();
+        }
+
+        return null;
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/556f309d/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/init/ControllerServiceInitializer.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/init/ControllerServiceInitializer.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/init/ControllerServiceInitializer.java
new file mode 100644
index 0000000..21b107f
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/init/ControllerServiceInitializer.java
@@ -0,0 +1,59 @@
+/*
+ * 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.init;
+
+import org.apache.nifi.annotation.lifecycle.OnShutdown;
+import org.apache.nifi.components.ConfigurableComponent;
+import org.apache.nifi.controller.ControllerService;
+import org.apache.nifi.controller.ControllerServiceInitializationContext;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.mock.MockComponentLogger;
+import org.apache.nifi.mock.MockConfigurationContext;
+import org.apache.nifi.mock.MockControllerServiceInitializationContext;
+import org.apache.nifi.nar.ExtensionManager;
+import org.apache.nifi.nar.NarCloseable;
+import org.apache.nifi.reporting.InitializationException;
+
+/**
+ * Initializes a ControllerService using a MockControllerServiceInitializationContext
+ *
+ *
+ */
+public class ControllerServiceInitializer implements ConfigurableComponentInitializer {
+
+    @Override
+    public void initialize(ConfigurableComponent component) throws InitializationException {
+        ControllerService controllerService = (ControllerService) component;
+        ControllerServiceInitializationContext context = new MockControllerServiceInitializationContext();
+        try (NarCloseable narCloseable = NarCloseable.withComponentNarLoader(component.getClass(), context.getIdentifier())) {
+            controllerService.initialize(context);
+        }
+    }
+
+    @Override
+    public void teardown(ConfigurableComponent component) {
+        try (NarCloseable narCloseable = NarCloseable.withComponentNarLoader(component.getClass(), component.getIdentifier())) {
+            ControllerService controllerService = (ControllerService) component;
+
+            final ComponentLog logger = new MockComponentLogger();
+            final MockConfigurationContext context = new MockConfigurationContext();
+            ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnShutdown.class, controllerService, logger, context);
+        } finally {
+            ExtensionManager.removeInstanceClassLoader(component.getIdentifier());
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/556f309d/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/init/ProcessorInitializer.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/init/ProcessorInitializer.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/init/ProcessorInitializer.java
new file mode 100644
index 0000000..06fdead
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/init/ProcessorInitializer.java
@@ -0,0 +1,58 @@
+/*
+ * 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.init;
+
+import org.apache.nifi.annotation.lifecycle.OnShutdown;
+import org.apache.nifi.components.ConfigurableComponent;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.mock.MockComponentLogger;
+import org.apache.nifi.mock.MockProcessContext;
+import org.apache.nifi.mock.MockProcessorInitializationContext;
+import org.apache.nifi.nar.ExtensionManager;
+import org.apache.nifi.nar.NarCloseable;
+import org.apache.nifi.processor.Processor;
+import org.apache.nifi.processor.ProcessorInitializationContext;
+
+/**
+ * Initializes a Processor using a MockProcessorInitializationContext
+ *
+ *
+ */
+public class ProcessorInitializer implements ConfigurableComponentInitializer {
+
+    @Override
+    public void initialize(ConfigurableComponent component) {
+        Processor processor = (Processor) component;
+        ProcessorInitializationContext initializationContext = new MockProcessorInitializationContext();
+        try (NarCloseable narCloseable = NarCloseable.withComponentNarLoader(component.getClass(), initializationContext.getIdentifier())) {
+            processor.initialize(initializationContext);
+        }
+    }
+
+    @Override
+    public void teardown(ConfigurableComponent component) {
+        Processor processor = (Processor) component;
+        try (NarCloseable narCloseable = NarCloseable.withComponentNarLoader(component.getClass(), component.getIdentifier())) {
+
+            final ComponentLog logger = new MockComponentLogger();
+            final MockProcessContext context = new MockProcessContext();
+            ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnShutdown.class, processor, logger, context);
+        } finally {
+            ExtensionManager.removeInstanceClassLoader(component.getIdentifier());
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/556f309d/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/init/ReflectionUtils.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/init/ReflectionUtils.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/init/ReflectionUtils.java
new file mode 100644
index 0000000..22420bd
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/init/ReflectionUtils.java
@@ -0,0 +1,133 @@
+/*
+ * 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.init;
+
+import org.apache.nifi.logging.ComponentLog;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.lang.annotation.Annotation;
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
+
+/**
+ * This class is a copy of org.apache.nifi.util.ReflectionUtils. Ultimately the
+ * documentation generation component should be moved to a place where it can
+ * depend on this directly instead of copying it in.
+ *
+ *
+ */
+public class ReflectionUtils {
+
+    private final static Logger LOG = LoggerFactory.getLogger(ReflectionUtils.class);
+
+    /**
+     * Invokes all methods on the given instance that have been annotated with
+     * the given annotation. If the signature of the method that is defined in
+     * <code>instance</code> uses 1 or more parameters, those parameters must be
+     * specified by the <code>args</code> parameter. However, if more arguments
+     * are supplied by the <code>args</code> parameter than needed, the extra
+     * arguments will be ignored.
+     *
+     * @param annotation annotation
+     * @param instance instance
+     * @param logger the ComponentLog to use for logging any errors. If null,
+     * will use own logger, but that will not generate bulletins or easily tie
+     * to the Processor's log messages.
+     * @param args args
+     * @return <code>true</code> if all appropriate methods were invoked and
+     * returned without throwing an Exception, <code>false</code> if one of the
+     * methods threw an Exception or could not be invoked; if <code>false</code>
+     * is returned, an error will have been logged.
+     */
+    public static boolean quietlyInvokeMethodsWithAnnotation(
+            final Class<? extends Annotation> annotation, final Object instance, final ComponentLog logger, final Object... args) {
+
+        for (final Method method : instance.getClass().getMethods()) {
+            if (method.isAnnotationPresent(annotation)) {
+
+                final boolean isAccessible = method.isAccessible();
+                method.setAccessible(true);
+
+                try {
+                    final Class<?>[] argumentTypes = method.getParameterTypes();
+                    if (argumentTypes.length > args.length) {
+                        if (logger == null) {
+                            LOG.error("Unable to invoke method {} on {} because method expects {} parameters but only {} were given",
+                                    new Object[]{method.getName(), instance, argumentTypes.length, args.length});
+                        } else {
+                            logger.error("Unable to invoke method {} on {} because method expects {} parameters but only {} were given",
+                                    new Object[]{method.getName(), instance, argumentTypes.length, args.length});
+                        }
+
+                        return false;
+                    }
+
+                    for (int i = 0; i < argumentTypes.length; i++) {
+                        final Class<?> argType = argumentTypes[i];
+                        if (!argType.isAssignableFrom(args[i].getClass())) {
+                            if (logger == null) {
+                                LOG.error("Unable to invoke method {} on {} because method parameter {} is expected to be of type {} but argument passed was of type {}",
+                                        new Object[]{method.getName(), instance, i, argType, args[i].getClass()});
+                            } else {
+                                logger.error("Unable to invoke method {} on {} because method parameter {} is expected to be of type {} but argument passed was of type {}",
+                                        new Object[]{method.getName(), instance, i, argType, args[i].getClass()});
+                            }
+
+                            return false;
+                        }
+                    }
+
+                    try {
+                        if (argumentTypes.length == args.length) {
+                            method.invoke(instance, args);
+                        } else {
+                            final Object[] argsToPass = new Object[argumentTypes.length];
+                            for (int i = 0; i < argsToPass.length; i++) {
+                                argsToPass[i] = args[i];
+                            }
+
+                            method.invoke(instance, argsToPass);
+                        }
+                    } catch (final InvocationTargetException ite) {
+                        if (logger == null) {
+                            LOG.error("Unable to invoke method {} on {} due to {}", new Object[]{method.getName(), instance, ite.getCause()});
+                            LOG.error("", ite.getCause());
+                        } else {
+                            logger.error("Unable to invoke method {} on {} due to {}", new Object[]{method.getName(), instance, ite.getCause()});
+                        }
+                    } catch (final IllegalAccessException | IllegalArgumentException t) {
+                        if (logger == null) {
+                            LOG.error("Unable to invoke method {} on {} due to {}", new Object[]{method.getName(), instance, t});
+                            LOG.error("", t);
+                        } else {
+                            logger.error("Unable to invoke method {} on {} due to {}", new Object[]{method.getName(), instance, t});
+                        }
+
+                        return false;
+                    }
+                } finally {
+                    if (!isAccessible) {
+                        method.setAccessible(false);
+                    }
+                }
+            }
+        }
+
+        return true;
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/556f309d/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/init/ReportingTaskingInitializer.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/init/ReportingTaskingInitializer.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/init/ReportingTaskingInitializer.java
new file mode 100644
index 0000000..f0f495d
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/init/ReportingTaskingInitializer.java
@@ -0,0 +1,57 @@
+/*
+ * 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.init;
+
+import org.apache.nifi.annotation.lifecycle.OnShutdown;
+import org.apache.nifi.components.ConfigurableComponent;
+import org.apache.nifi.mock.MockComponentLogger;
+import org.apache.nifi.mock.MockConfigurationContext;
+import org.apache.nifi.mock.MockReportingInitializationContext;
+import org.apache.nifi.nar.ExtensionManager;
+import org.apache.nifi.nar.NarCloseable;
+import org.apache.nifi.reporting.InitializationException;
+import org.apache.nifi.reporting.ReportingInitializationContext;
+import org.apache.nifi.reporting.ReportingTask;
+
+/**
+ * Initializes a ReportingTask using a MockReportingInitializationContext;
+ *
+ *
+ */
+public class ReportingTaskingInitializer implements ConfigurableComponentInitializer {
+
+    @Override
+    public void initialize(ConfigurableComponent component) throws InitializationException {
+        ReportingTask reportingTask = (ReportingTask) component;
+        ReportingInitializationContext context = new MockReportingInitializationContext();
+        try (NarCloseable narCloseable = NarCloseable.withComponentNarLoader(component.getClass(), context.getIdentifier())) {
+            reportingTask.initialize(context);
+        }
+    }
+
+    @Override
+    public void teardown(ConfigurableComponent component) {
+        ReportingTask reportingTask = (ReportingTask) component;
+        try (NarCloseable narCloseable = NarCloseable.withComponentNarLoader(component.getClass(), component.getIdentifier())) {
+
+            final MockConfigurationContext context = new MockConfigurationContext();
+            ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnShutdown.class, reportingTask, new MockComponentLogger(), context);
+        } finally {
+            ExtensionManager.removeInstanceClassLoader(component.getIdentifier());
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/556f309d/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/mock/MockComponentLogger.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/mock/MockComponentLogger.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/mock/MockComponentLogger.java
new file mode 100644
index 0000000..920d7eb
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/mock/MockComponentLogger.java
@@ -0,0 +1,258 @@
+/*
+ * 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.mock;
+
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.logging.LogLevel;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Stubs out the functionality of a ComponentLog so that it can
+ * be used during initialization of a component.
+ *
+ */
+public class MockComponentLogger implements ComponentLog {
+
+    private static final Logger logger = LoggerFactory
+            .getLogger(MockComponentLogger.class);
+
+    @Override
+    public void warn(String msg, Throwable t) {
+        logger.warn(msg, t);
+    }
+
+    @Override
+    public void warn(String msg, Object[] os) {
+        logger.warn(msg, os);
+    }
+
+    @Override
+    public void warn(String msg, Object[] os, Throwable t) {
+        logger.warn(msg, os);
+        logger.warn("", t);
+    }
+
+    @Override
+    public void warn(String msg) {
+        logger.warn(msg);
+    }
+
+    @Override
+    public void trace(String msg, Throwable t) {
+        logger.trace(msg, t);
+    }
+
+    @Override
+    public void trace(String msg, Object[] os) {
+        logger.trace(msg, os);
+    }
+
+    @Override
+    public void trace(String msg) {
+        logger.trace(msg);
+    }
+
+    @Override
+    public void trace(String msg, Object[] os, Throwable t) {
+        logger.trace(msg, os);
+        logger.trace("", t);
+    }
+
+    @Override
+    public boolean isWarnEnabled() {
+        return logger.isWarnEnabled();
+    }
+
+    @Override
+    public boolean isTraceEnabled() {
+        return logger.isTraceEnabled();
+    }
+
+    @Override
+    public boolean isInfoEnabled() {
+        return logger.isInfoEnabled();
+    }
+
+    @Override
+    public boolean isErrorEnabled() {
+        return logger.isErrorEnabled();
+    }
+
+    @Override
+    public boolean isDebugEnabled() {
+        return logger.isDebugEnabled();
+    }
+
+    @Override
+    public void info(String msg, Throwable t) {
+        logger.info(msg, t);
+    }
+
+    @Override
+    public void info(String msg, Object[] os) {
+        logger.info(msg, os);
+    }
+
+    @Override
+    public void info(String msg) {
+        logger.info(msg);
+
+    }
+
+    @Override
+    public void info(String msg, Object[] os, Throwable t) {
+        logger.trace(msg, os);
+        logger.trace("", t);
+
+    }
+
+    @Override
+    public String getName() {
+        return logger.getName();
+    }
+
+    @Override
+    public void error(String msg, Throwable t) {
+        logger.error(msg, t);
+    }
+
+    @Override
+    public void error(String msg, Object[] os) {
+        logger.error(msg, os);
+    }
+
+    @Override
+    public void error(String msg) {
+        logger.error(msg);
+    }
+
+    @Override
+    public void error(String msg, Object[] os, Throwable t) {
+        logger.error(msg, os);
+        logger.error("", t);
+    }
+
+    @Override
+    public void debug(String msg, Throwable t) {
+        logger.debug(msg, t);
+    }
+
+    @Override
+    public void debug(String msg, Object[] os) {
+        logger.debug(msg, os);
+    }
+
+    @Override
+    public void debug(String msg, Object[] os, Throwable t) {
+        logger.debug(msg, os);
+        logger.debug("", t);
+    }
+
+    @Override
+    public void debug(String msg) {
+        logger.debug(msg);
+    }
+
+    @Override
+    public void log(LogLevel level, String msg, Throwable t) {
+        switch (level) {
+            case DEBUG:
+                debug(msg, t);
+                break;
+            case ERROR:
+            case FATAL:
+                error(msg, t);
+                break;
+            case INFO:
+                info(msg, t);
+                break;
+            case TRACE:
+                trace(msg, t);
+                break;
+            case WARN:
+                warn(msg, t);
+                break;
+        }
+    }
+
+    @Override
+    public void log(LogLevel level, String msg, Object[] os) {
+        switch (level) {
+            case DEBUG:
+                debug(msg, os);
+                break;
+            case ERROR:
+            case FATAL:
+                error(msg, os);
+                break;
+            case INFO:
+                info(msg, os);
+                break;
+            case TRACE:
+                trace(msg, os);
+                break;
+            case WARN:
+                warn(msg, os);
+                break;
+        }
+    }
+
+    @Override
+    public void log(LogLevel level, String msg) {
+        switch (level) {
+            case DEBUG:
+                debug(msg);
+                break;
+            case ERROR:
+            case FATAL:
+                error(msg);
+                break;
+            case INFO:
+                info(msg);
+                break;
+            case TRACE:
+                trace(msg);
+                break;
+            case WARN:
+                warn(msg);
+                break;
+        }
+    }
+
+    @Override
+    public void log(LogLevel level, String msg, Object[] os, Throwable t) {
+        switch (level) {
+            case DEBUG:
+                debug(msg, os, t);
+                break;
+            case ERROR:
+            case FATAL:
+                error(msg, os, t);
+                break;
+            case INFO:
+                info(msg, os, t);
+                break;
+            case TRACE:
+                trace(msg, os, t);
+                break;
+            case WARN:
+                warn(msg, os, t);
+                break;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/556f309d/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/mock/MockConfigurationContext.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/mock/MockConfigurationContext.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/mock/MockConfigurationContext.java
new file mode 100644
index 0000000..d1e73fb
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/mock/MockConfigurationContext.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.mock;
+
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.PropertyValue;
+import org.apache.nifi.controller.ConfigurationContext;
+
+import java.util.Collections;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+public class MockConfigurationContext implements ConfigurationContext {
+
+    @Override
+    public PropertyValue getProperty(PropertyDescriptor property) {
+        return null;
+    }
+
+    @Override
+    public Map<PropertyDescriptor, String> getProperties() {
+        return Collections.emptyMap();
+    }
+
+    @Override
+    public String getSchedulingPeriod() {
+        return "0 secs";
+    }
+
+    @Override
+    public Long getSchedulingPeriod(final TimeUnit timeUnit) {
+        return 0L;
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/556f309d/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/mock/MockControllerServiceInitializationContext.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/mock/MockControllerServiceInitializationContext.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/mock/MockControllerServiceInitializationContext.java
new file mode 100644
index 0000000..b111ad2
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/mock/MockControllerServiceInitializationContext.java
@@ -0,0 +1,68 @@
+/*
+ * 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.mock;
+
+import org.apache.nifi.components.state.StateManager;
+import org.apache.nifi.controller.ControllerServiceInitializationContext;
+import org.apache.nifi.controller.ControllerServiceLookup;
+import org.apache.nifi.logging.ComponentLog;
+
+import java.io.File;
+
+/**
+ * A Mock ControllerServiceInitializationContext so that ControllerServices can
+ * be initialized for the purpose of generating documentation.
+ *
+ *
+ */
+public class MockControllerServiceInitializationContext implements ControllerServiceInitializationContext {
+
+    @Override
+    public String getIdentifier() {
+        return "mock-controller-service";
+    }
+
+    @Override
+    public ControllerServiceLookup getControllerServiceLookup() {
+        return new MockControllerServiceLookup();
+    }
+
+    @Override
+    public ComponentLog getLogger() {
+        return new MockComponentLogger();
+    }
+
+    @Override
+    public StateManager getStateManager() {
+        return null;
+    }
+
+    @Override
+    public String getKerberosServicePrincipal() {
+        return null;
+    }
+
+    @Override
+    public File getKerberosServiceKeytab() {
+        return null;
+    }
+
+    @Override
+    public File getKerberosConfigurationFile() {
+        return null;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi/blob/556f309d/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/mock/MockControllerServiceLookup.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/mock/MockControllerServiceLookup.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/mock/MockControllerServiceLookup.java
new file mode 100644
index 0000000..5307ac4
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/mock/MockControllerServiceLookup.java
@@ -0,0 +1,63 @@
+/*
+ * 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.mock;
+
+import org.apache.nifi.controller.ControllerService;
+import org.apache.nifi.controller.ControllerServiceLookup;
+
+import java.util.Collections;
+import java.util.Set;
+
+/**
+ * A Mock ControllerServiceLookup that can be used so that
+ * ConfigurableComponents can be initialized for the purpose of generating
+ * documentation
+ *
+ *
+ */
+public class MockControllerServiceLookup implements ControllerServiceLookup {
+
+    @Override
+    public ControllerService getControllerService(final String serviceIdentifier) {
+        return null;
+    }
+
+    @Override
+    public boolean isControllerServiceEnabled(final String serviceIdentifier) {
+        return false;
+    }
+
+    @Override
+    public boolean isControllerServiceEnabled(final ControllerService service) {
+        return false;
+    }
+
+    @Override
+    public Set<String> getControllerServiceIdentifiers(final Class<? extends ControllerService> serviceType) throws IllegalArgumentException {
+        return Collections.emptySet();
+    }
+
+    @Override
+    public boolean isControllerServiceEnabling(final String serviceIdentifier) {
+        return false;
+    }
+
+    @Override
+    public String getControllerServiceName(final String serviceIdentifier) {
+        return serviceIdentifier;
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/556f309d/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/mock/MockNodeTypeProvider.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/mock/MockNodeTypeProvider.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/mock/MockNodeTypeProvider.java
new file mode 100644
index 0000000..61390e1
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/mock/MockNodeTypeProvider.java
@@ -0,0 +1,40 @@
+/*
+ * 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.mock;
+
+import org.apache.nifi.controller.NodeTypeProvider;
+
+/**
+ * A Mock NodeTypeProvider that can be used so that
+ * ConfigurableComponents can be initialized for the purpose of generating
+ * documentation
+ *
+ *
+ */
+public class MockNodeTypeProvider implements NodeTypeProvider {
+
+    @Override
+    public boolean isClustered() {
+        return false;
+    }
+
+    @Override
+    public boolean isPrimary() {
+        return false;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/556f309d/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/mock/MockProcessContext.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/mock/MockProcessContext.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/mock/MockProcessContext.java
new file mode 100644
index 0000000..cf2e2cf
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/mock/MockProcessContext.java
@@ -0,0 +1,116 @@
+/*
+ * 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.mock;
+
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.PropertyValue;
+import org.apache.nifi.components.state.StateManager;
+import org.apache.nifi.controller.ControllerServiceLookup;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.Relationship;
+
+import java.util.Collections;
+import java.util.Map;
+import java.util.Set;
+
+public class MockProcessContext implements ProcessContext {
+
+    @Override
+    public PropertyValue getProperty(PropertyDescriptor descriptor) {
+        return null;
+    }
+
+    @Override
+    public PropertyValue getProperty(String propertyName) {
+        return null;
+    }
+
+    @Override
+    public PropertyValue newPropertyValue(String rawValue) {
+        return null;
+    }
+
+    @Override
+    public void yield() {
+
+    }
+
+    @Override
+    public int getMaxConcurrentTasks() {
+        return 0;
+    }
+
+    @Override
+    public String getAnnotationData() {
+        return "";
+    }
+
+    @Override
+    public Map<PropertyDescriptor, String> getProperties() {
+        return Collections.emptyMap();
+    }
+
+    @Override
+    public String encrypt(String unencrypted) {
+        return unencrypted;
+    }
+
+    @Override
+    public String decrypt(String encrypted) {
+        return encrypted;
+    }
+
+    @Override
+    public ControllerServiceLookup getControllerServiceLookup() {
+        return new MockControllerServiceLookup();
+    }
+
+    @Override
+    public Set<Relationship> getAvailableRelationships() {
+        return Collections.emptySet();
+    }
+
+    @Override
+    public boolean hasIncomingConnection() {
+        return true;
+    }
+
+    @Override
+    public boolean hasNonLoopConnection() {
+        return true;
+    }
+
+    @Override
+    public boolean hasConnection(Relationship relationship) {
+        return false;
+    }
+
+    @Override
+    public boolean isExpressionLanguagePresent(PropertyDescriptor property) {
+        return false;
+    }
+
+    @Override
+    public StateManager getStateManager() {
+        return null;
+    }
+
+    @Override
+    public String getName() {
+        return null;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi/blob/556f309d/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/mock/MockProcessorInitializationContext.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/mock/MockProcessorInitializationContext.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/mock/MockProcessorInitializationContext.java
new file mode 100644
index 0000000..d9320b2
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/mock/MockProcessorInitializationContext.java
@@ -0,0 +1,68 @@
+/*
+ * 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.mock;
+
+import org.apache.nifi.controller.ControllerServiceLookup;
+import org.apache.nifi.controller.NodeTypeProvider;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.processor.ProcessorInitializationContext;
+
+import java.io.File;
+
+/**
+ * A Mock ProcessorInitializationContext that can be used so that Processors can
+ * be initialized for the purpose of generating documentation.
+ *
+ *
+ */
+public class MockProcessorInitializationContext implements ProcessorInitializationContext {
+
+    @Override
+    public String getIdentifier() {
+        return "mock-processor";
+    }
+
+    @Override
+    public ComponentLog getLogger() {
+        return new MockComponentLogger();
+    }
+
+    @Override
+    public ControllerServiceLookup getControllerServiceLookup() {
+        return new MockControllerServiceLookup();
+    }
+
+    @Override
+    public NodeTypeProvider getNodeTypeProvider() {
+        return new MockNodeTypeProvider();
+    }
+
+    @Override
+    public String getKerberosServicePrincipal() {
+        return null;
+    }
+
+    @Override
+    public File getKerberosServiceKeytab() {
+        return null;
+    }
+
+    @Override
+    public File getKerberosConfigurationFile() {
+        return null;
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/556f309d/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/mock/MockReportingInitializationContext.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/mock/MockReportingInitializationContext.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/mock/MockReportingInitializationContext.java
new file mode 100644
index 0000000..630c657
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/mock/MockReportingInitializationContext.java
@@ -0,0 +1,83 @@
+/*
+ * 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.mock;
+
+import org.apache.nifi.controller.ControllerServiceLookup;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.reporting.ReportingInitializationContext;
+import org.apache.nifi.scheduling.SchedulingStrategy;
+
+import java.io.File;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * A Mock ReportingInitializationContext that can be used to initialize a
+ * ReportingTask for the purposes of documentation generation.
+ *
+ */
+public class MockReportingInitializationContext implements ReportingInitializationContext {
+
+    @Override
+    public String getIdentifier() {
+        return "mock-reporting-task";
+    }
+
+    @Override
+    public String getName() {
+        return "";
+    }
+
+    @Override
+    public long getSchedulingPeriod(TimeUnit timeUnit) {
+        return 0;
+    }
+
+    @Override
+    public ControllerServiceLookup getControllerServiceLookup() {
+        return new MockControllerServiceLookup();
+    }
+
+    @Override
+    public String getSchedulingPeriod() {
+        return "";
+    }
+
+    @Override
+    public SchedulingStrategy getSchedulingStrategy() {
+        return SchedulingStrategy.TIMER_DRIVEN;
+    }
+
+    @Override
+    public ComponentLog getLogger() {
+        return new MockComponentLogger();
+    }
+
+    @Override
+    public String getKerberosServicePrincipal() {
+        return null;
+    }
+
+    @Override
+    public File getKerberosServiceKeytab() {
+        return null;
+    }
+
+    @Override
+    public File getKerberosConfigurationFile() {
+        return null;
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/556f309d/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/nar/ExtensionManager.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/nar/ExtensionManager.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/nar/ExtensionManager.java
new file mode 100644
index 0000000..14d3dcc
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/nar/ExtensionManager.java
@@ -0,0 +1,537 @@
+/*
+ * 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 org.apache.nifi.annotation.behavior.RequiresInstanceClassLoading;
+import org.apache.nifi.authentication.LoginIdentityProvider;
+import org.apache.nifi.authorization.Authorizer;
+import org.apache.nifi.bundle.Bundle;
+import org.apache.nifi.bundle.BundleCoordinate;
+import org.apache.nifi.components.ConfigurableComponent;
+import org.apache.nifi.components.PropertyDescriptor;
+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;
+import org.apache.nifi.controller.repository.FlowFileSwapManager;
+import org.apache.nifi.controller.status.history.ComponentStatusRepository;
+import org.apache.nifi.flowfile.FlowFilePrioritizer;
+import org.apache.nifi.init.ConfigurableComponentInitializer;
+import org.apache.nifi.init.ConfigurableComponentInitializerFactory;
+import org.apache.nifi.processor.Processor;
+import org.apache.nifi.provenance.ProvenanceRepository;
+import org.apache.nifi.reporting.InitializationException;
+import org.apache.nifi.reporting.ReportingTask;
+import org.apache.nifi.util.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.net.URL;
+import java.net.URLClassLoader;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.ServiceLoader;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.stream.Collectors;
+
+/**
+ * Scans through the classpath to load all FlowFileProcessors, FlowFileComparators, and ReportingTasks using the service provider API and running through all classloaders (root, NARs).
+ *
+ * @ThreadSafe - is immutable
+ */
+@SuppressWarnings("rawtypes")
+public class ExtensionManager {
+
+    private static final Logger logger = LoggerFactory.getLogger(ExtensionManager.class);
+
+    // Maps a service definition (interface) to those classes that implement the interface
+    private static final Map<Class, Set<Class>> definitionMap = new HashMap<>();
+
+    private static final Map<String, List<Bundle>> classNameBundleLookup = new HashMap<>();
+    private static final Map<BundleCoordinate, Bundle> bundleCoordinateBundleLookup = new HashMap<>();
+    private static final Map<ClassLoader, Bundle> classLoaderBundleLookup = new HashMap<>();
+    private static final Map<String, ConfigurableComponent> tempComponentLookup = new HashMap<>();
+
+    private static final Map<String, Class<?>> requiresInstanceClassLoading = new HashMap<>();
+    private static final Map<String, InstanceClassLoader> instanceClassloaderLookup = new ConcurrentHashMap<>();
+
+    static {
+        definitionMap.put(Processor.class, new HashSet<>());
+        definitionMap.put(FlowFilePrioritizer.class, new HashSet<>());
+        definitionMap.put(ReportingTask.class, new HashSet<>());
+        definitionMap.put(ControllerService.class, new HashSet<>());
+        definitionMap.put(Authorizer.class, new HashSet<>());
+        definitionMap.put(LoginIdentityProvider.class, new HashSet<>());
+        definitionMap.put(ProvenanceRepository.class, new HashSet<>());
+        definitionMap.put(ComponentStatusRepository.class, new HashSet<>());
+        definitionMap.put(FlowFileRepository.class, new HashSet<>());
+        definitionMap.put(FlowFileSwapManager.class, new HashSet<>());
+        definitionMap.put(ContentRepository.class, new HashSet<>());
+        definitionMap.put(StateProvider.class, new HashSet<>());
+    }
+
+    /**
+     * Loads all FlowFileProcessor, FlowFileComparator, ReportingTask class types that can be found on the bootstrap classloader and by creating classloaders for all NARs found within the classpath.
+     * @param narBundles the bundles to scan through in search of extensions
+     */
+    public static void discoverExtensions(final Bundle systemBundle, final Set<Bundle> narBundles) {
+        // get the current context class loader
+        ClassLoader currentContextClassLoader = Thread.currentThread().getContextClassLoader();
+
+        // load the system bundle first so that any extensions found in JARs directly in lib will be registered as
+        // being from the system bundle and not from all the other NARs
+        loadExtensions(systemBundle);
+        bundleCoordinateBundleLookup.put(systemBundle.getBundleDetails().getCoordinate(), systemBundle);
+
+        // consider each nar class loader
+        for (final Bundle bundle : narBundles) {
+            // Must set the context class loader to the nar classloader itself
+            // so that static initialization techniques that depend on the context class loader will work properly
+            final ClassLoader ncl = bundle.getClassLoader();
+            Thread.currentThread().setContextClassLoader(ncl);
+            loadExtensions(bundle);
+
+            // Create a look-up from coordinate to bundle
+            bundleCoordinateBundleLookup.put(bundle.getBundleDetails().getCoordinate(), bundle);
+        }
+
+        // restore the current context class loader if appropriate
+        if (currentContextClassLoader != null) {
+            Thread.currentThread().setContextClassLoader(currentContextClassLoader);
+        }
+    }
+
+    /**
+     * Loads extensions from the specified bundle.
+     *
+     * @param bundle from which to load extensions
+     */
+    @SuppressWarnings("unchecked")
+    private static void loadExtensions(final Bundle bundle) {
+        for (final Map.Entry<Class, Set<Class>> entry : definitionMap.entrySet()) {
+            final boolean isControllerService = ControllerService.class.equals(entry.getKey());
+            final boolean isProcessor = Processor.class.equals(entry.getKey());
+            final boolean isReportingTask = ReportingTask.class.equals(entry.getKey());
+
+            final ServiceLoader<?> serviceLoader = ServiceLoader.load(entry.getKey(), bundle.getClassLoader());
+            for (final Object o : serviceLoader) {
+                // create a cache of temp ConfigurableComponent instances, the initialize here has to happen before the checks below
+                if ((isControllerService || isProcessor || isReportingTask) && o instanceof ConfigurableComponent) {
+                    final ConfigurableComponent configurableComponent = (ConfigurableComponent) o;
+                    initializeTempComponent(configurableComponent);
+
+                    final String cacheKey = getClassBundleKey(o.getClass().getCanonicalName(), bundle.getBundleDetails().getCoordinate());
+                    tempComponentLookup.put(cacheKey, (ConfigurableComponent)o);
+                }
+
+                // only consider extensions discovered directly in this bundle
+                boolean registerExtension = bundle.getClassLoader().equals(o.getClass().getClassLoader());
+
+                if (registerExtension) {
+                    final Class extensionType = o.getClass();
+                    if (isControllerService && !checkControllerServiceEligibility(extensionType)) {
+                        registerExtension = false;
+                        logger.error(String.format(
+                                "Skipping Controller Service %s because it is bundled with its supporting APIs and requires instance class loading.", extensionType.getName()));
+                    }
+
+                    final boolean canReferenceControllerService = (isControllerService || isProcessor || isReportingTask) && o instanceof ConfigurableComponent;
+                    if (canReferenceControllerService && !checkControllerServiceReferenceEligibility((ConfigurableComponent) o, bundle.getClassLoader())) {
+                        registerExtension = false;
+                        logger.error(String.format(
+                                "Skipping component %s because it is bundled with its referenced Controller Service APIs and requires instance class loading.", extensionType.getName()));
+                    }
+
+                    if (registerExtension) {
+                        registerServiceClass(o.getClass(), classNameBundleLookup, bundle, entry.getValue());
+                    }
+                }
+
+            }
+
+            classLoaderBundleLookup.put(bundle.getClassLoader(), bundle);
+        }
+    }
+
+    private static void initializeTempComponent(final ConfigurableComponent configurableComponent) {
+        ConfigurableComponentInitializer initializer = null;
+        try {
+            initializer = ConfigurableComponentInitializerFactory.createComponentInitializer(configurableComponent.getClass());
+            initializer.initialize(configurableComponent);
+        } catch (final InitializationException e) {
+            logger.warn(String.format("Unable to initialize component %s due to %s", configurableComponent.getClass().getName(), e.getMessage()));
+        }
+    }
+
+    private static boolean checkControllerServiceReferenceEligibility(final ConfigurableComponent component, final ClassLoader classLoader) {
+        // if the extension does not require instance classloading, its eligible
+        final boolean requiresInstanceClassLoading = component.getClass().isAnnotationPresent(RequiresInstanceClassLoading.class);
+
+        final Set<Class> cobundledApis = new HashSet<>();
+        try (final NarCloseable closeable = NarCloseable.withComponentNarLoader(component.getClass().getClassLoader())) {
+            final List<PropertyDescriptor> descriptors = component.getPropertyDescriptors();
+            if (descriptors != null && !descriptors.isEmpty()) {
+                for (final PropertyDescriptor descriptor : descriptors) {
+                    final Class<? extends ControllerService> serviceApi = descriptor.getControllerServiceDefinition();
+                    if (serviceApi != null && classLoader.equals(serviceApi.getClassLoader())) {
+                        cobundledApis.add(serviceApi);
+                    }
+                }
+            }
+        }
+
+        if (!cobundledApis.isEmpty()) {
+            logger.warn(String.format(
+                    "Component %s is bundled with its referenced Controller Service APIs %s. The service APIs should not be bundled with component implementations that reference it.",
+                    component.getClass().getName(), StringUtils.join(cobundledApis.stream().map(cls -> cls.getName()).collect(Collectors.toSet()), ", ")));
+        }
+
+        // the component is eligible when it does not require instance classloading or when the supporting APIs are bundled in a parent NAR
+        return requiresInstanceClassLoading == false || cobundledApis.isEmpty();
+    }
+
+    private static boolean checkControllerServiceEligibility(Class extensionType) {
+        final Class originalExtensionType = extensionType;
+        final ClassLoader originalExtensionClassLoader = extensionType.getClassLoader();
+
+        // if the extension does not require instance classloading, its eligible
+        final boolean requiresInstanceClassLoading = extensionType.isAnnotationPresent(RequiresInstanceClassLoading.class);
+
+        final Set<Class> cobundledApis = new HashSet<>();
+        while (extensionType != null) {
+            for (final Class i : extensionType.getInterfaces()) {
+                if (originalExtensionClassLoader.equals(i.getClassLoader())) {
+                    cobundledApis.add(i);
+                }
+            }
+
+            extensionType = extensionType.getSuperclass();
+        }
+
+        if (!cobundledApis.isEmpty()) {
+            logger.warn(String.format("Controller Service %s is bundled with its supporting APIs %s. The service APIs should not be bundled with the implementations.",
+                    originalExtensionType.getName(), StringUtils.join(cobundledApis.stream().map(cls -> cls.getName()).collect(Collectors.toSet()), ", ")));
+        }
+
+        // the service is eligible when it does not require instance classloading or when the supporting APIs are bundled in a parent NAR
+        return requiresInstanceClassLoading == false || cobundledApis.isEmpty();
+    }
+
+    /**
+     * Registers extension for the specified type from the specified Bundle.
+     *
+     * @param type the extension type
+     * @param classNameBundleMap mapping of classname to Bundle
+     * @param bundle the Bundle being mapped to
+     * @param classes to map to this classloader but which come from its ancestors
+     */
+    private static void registerServiceClass(final Class<?> type, final Map<String, List<Bundle>> classNameBundleMap, final Bundle bundle, final Set<Class> classes) {
+        final String className = type.getName();
+
+        // get the bundles that have already been registered for the class name
+        List<Bundle> registeredBundles = classNameBundleMap.get(className);
+
+        if (registeredBundles == null) {
+            registeredBundles = new ArrayList<>();
+            classNameBundleMap.put(className, registeredBundles);
+        }
+
+        boolean alreadyRegistered = false;
+        for (final Bundle registeredBundle : registeredBundles) {
+            final BundleCoordinate registeredCoordinate = registeredBundle.getBundleDetails().getCoordinate();
+
+            // if the incoming bundle has the same coordinate as one of the registered bundles then consider it already registered
+            if (registeredCoordinate.equals(bundle.getBundleDetails().getCoordinate())) {
+                alreadyRegistered = true;
+                break;
+            }
+
+            // if the type wasn't loaded from an ancestor, and the type isn't a processor, cs, or reporting task, then
+            // fail registration because we don't support multiple versions of any other types
+            if (!multipleVersionsAllowed(type)) {
+                throw new IllegalStateException("Attempt was made to load " + className + " from "
+                        + bundle.getBundleDetails().getCoordinate().getCoordinate()
+                        + " but that class name is already loaded/registered from " + registeredBundle.getBundleDetails().getCoordinate()
+                        + " and multiple versions are not supported for this type"
+                );
+            }
+        }
+
+        // if none of the above was true then register the new bundle
+        if (!alreadyRegistered) {
+            registeredBundles.add(bundle);
+            classes.add(type);
+
+            if (type.isAnnotationPresent(RequiresInstanceClassLoading.class)) {
+                final String cacheKey = getClassBundleKey(className, bundle.getBundleDetails().getCoordinate());
+                requiresInstanceClassLoading.put(cacheKey, type);
+            }
+        }
+
+    }
+
+    /**
+     * @param type a Class that we found from a service loader
+     * @return true if the given class is a processor, controller service, or reporting task
+     */
+    private static boolean multipleVersionsAllowed(Class<?> type) {
+        return Processor.class.isAssignableFrom(type) || ControllerService.class.isAssignableFrom(type) || ReportingTask.class.isAssignableFrom(type);
+    }
+
+    /**
+     * Determines the effective ClassLoader for the instance of the given type.
+     *
+     * @param classType the type of class to lookup the ClassLoader for
+     * @param instanceIdentifier the identifier of the specific instance of the classType to look up the ClassLoader for
+     * @param bundle the bundle where the classType exists
+     * @param additionalUrls additional URLs to add to the instance class loader
+     * @return the ClassLoader for the given instance of the given type, or null if the type is not a detected extension type
+     */
+    public static InstanceClassLoader createInstanceClassLoader(final String classType, final String instanceIdentifier, final Bundle bundle, final Set<URL> additionalUrls) {
+        if (StringUtils.isEmpty(classType)) {
+            throw new IllegalArgumentException("Class-Type is required");
+        }
+
+        if (StringUtils.isEmpty(instanceIdentifier)) {
+            throw new IllegalArgumentException("Instance Identifier is required");
+        }
+
+        if (bundle == null) {
+            throw new IllegalArgumentException("Bundle is required");
+        }
+
+        // If the class is annotated with @RequiresInstanceClassLoading and the registered ClassLoader is a URLClassLoader
+        // then make a new InstanceClassLoader that is a full copy of the NAR Class Loader, otherwise create an empty
+        // InstanceClassLoader that has the NAR ClassLoader as a parent
+
+        InstanceClassLoader instanceClassLoader;
+        final ClassLoader bundleClassLoader = bundle.getClassLoader();
+        final String key = getClassBundleKey(classType, bundle.getBundleDetails().getCoordinate());
+
+        if (requiresInstanceClassLoading.containsKey(key) && bundleClassLoader instanceof NarClassLoader) {
+            final Class<?> type = requiresInstanceClassLoading.get(key);
+            final RequiresInstanceClassLoading requiresInstanceClassLoading = type.getAnnotation(RequiresInstanceClassLoading.class);
+
+            final NarClassLoader narBundleClassLoader = (NarClassLoader) bundleClassLoader;
+            logger.debug("Including ClassLoader resources from {} for component {}", new Object[] {bundle.getBundleDetails(), instanceIdentifier});
+
+            final Set<URL> instanceUrls = new LinkedHashSet<>();
+            for (final URL url : narBundleClassLoader.getURLs()) {
+                instanceUrls.add(url);
+            }
+
+            ClassLoader ancestorClassLoader = narBundleClassLoader.getParent();
+
+            if (requiresInstanceClassLoading.cloneAncestorResources()) {
+                final ConfigurableComponent component = getTempComponent(classType, bundle.getBundleDetails().getCoordinate());
+                final Set<BundleCoordinate> reachableApiBundles = findReachableApiBundles(component);
+
+                while (ancestorClassLoader != null && ancestorClassLoader instanceof NarClassLoader) {
+                    final Bundle ancestorNarBundle = classLoaderBundleLookup.get(ancestorClassLoader);
+
+                    // stop including ancestor resources when we reach one of the APIs, or when we hit the Jetty NAR
+                    if (ancestorNarBundle == null || reachableApiBundles.contains(ancestorNarBundle.getBundleDetails().getCoordinate())
+                            || ancestorNarBundle.getBundleDetails().getCoordinate().getId().equals(NarClassLoaders.JETTY_NAR_ID)) {
+                        break;
+                    }
+
+                    final NarClassLoader ancestorNarClassLoader = (NarClassLoader) ancestorClassLoader;
+                    for (final URL url : ancestorNarClassLoader.getURLs()) {
+                        instanceUrls.add(url);
+                    }
+                    ancestorClassLoader = ancestorNarClassLoader.getParent();
+                }
+            }
+
+            instanceClassLoader = new InstanceClassLoader(instanceIdentifier, classType, instanceUrls, additionalUrls, ancestorClassLoader);
+        } else {
+            instanceClassLoader = new InstanceClassLoader(instanceIdentifier, classType, Collections.emptySet(), additionalUrls, bundleClassLoader);
+        }
+
+        if (logger.isTraceEnabled()) {
+            for (URL url : instanceClassLoader.getURLs()) {
+                logger.trace("URL resource {} for {}...", new Object[]{url.toExternalForm(), instanceIdentifier});
+            }
+        }
+
+        instanceClassloaderLookup.put(instanceIdentifier, instanceClassLoader);
+        return instanceClassLoader;
+    }
+
+    /**
+     * Find the bundle coordinates for any service APIs that are referenced by this component and not part of the same bundle.
+     *
+     * @param component the component being instantiated
+     */
+    protected static Set<BundleCoordinate> findReachableApiBundles(final ConfigurableComponent component) {
+        final Set<BundleCoordinate> reachableApiBundles = new HashSet<>();
+
+        try (final NarCloseable closeable = NarCloseable.withComponentNarLoader(component.getClass().getClassLoader())) {
+            final List<PropertyDescriptor> descriptors = component.getPropertyDescriptors();
+            if (descriptors != null && !descriptors.isEmpty()) {
+                for (final PropertyDescriptor descriptor : descriptors) {
+                    final Class<? extends ControllerService> serviceApi = descriptor.getControllerServiceDefinition();
+                    if (serviceApi != null && !component.getClass().getClassLoader().equals(serviceApi.getClassLoader())) {
+                        final Bundle apiBundle = classLoaderBundleLookup.get(serviceApi.getClassLoader());
+                        reachableApiBundles.add(apiBundle.getBundleDetails().getCoordinate());
+                    }
+                }
+            }
+        }
+
+        return reachableApiBundles;
+    }
+
+    /**
+     * Retrieves the InstanceClassLoader for the component with the given identifier.
+     *
+     * @param instanceIdentifier the identifier of a component
+     * @return the instance class loader for the component
+     */
+    public static InstanceClassLoader getInstanceClassLoader(final String instanceIdentifier) {
+        return instanceClassloaderLookup.get(instanceIdentifier);
+    }
+
+    /**
+     * Removes the InstanceClassLoader for a given component.
+     *
+     * @param instanceIdentifier the of a component
+     */
+    public static InstanceClassLoader removeInstanceClassLoader(final String instanceIdentifier) {
+        if (instanceIdentifier == null) {
+            return null;
+        }
+
+        final InstanceClassLoader classLoader = instanceClassloaderLookup.remove(instanceIdentifier);
+        closeURLClassLoader(instanceIdentifier, classLoader);
+        return classLoader;
+    }
+
+    /**
+     * Closes the given ClassLoader if it is an instance of URLClassLoader.
+     *
+     * @param instanceIdentifier the instance id the class loader corresponds to
+     * @param classLoader the class loader to close
+     */
+    public static void closeURLClassLoader(final String instanceIdentifier, final ClassLoader classLoader) {
+        if (classLoader != null && (classLoader instanceof URLClassLoader)) {
+            final URLClassLoader urlClassLoader = (URLClassLoader) classLoader;
+            try {
+                urlClassLoader.close();
+            } catch (IOException e) {
+                logger.warn("Unable to close URLClassLoader for " + instanceIdentifier);
+            }
+        }
+    }
+
+    /**
+     * Retrieves the bundles that have a class with the given name.
+     *
+     * @param classType the class name of an extension
+     * @return the list of bundles that contain an extension with the given class name
+     */
+    public static List<Bundle> getBundles(final String classType) {
+        if (classType == null) {
+            throw new IllegalArgumentException("Class type cannot be null");
+        }
+        final List<Bundle> bundles = classNameBundleLookup.get(classType);
+        return bundles == null ? Collections.emptyList() : new ArrayList<>(bundles);
+    }
+
+    /**
+     * Retrieves the bundle with the given coordinate.
+     *
+     * @param bundleCoordinate a coordinate to look up
+     * @return the bundle with the given coordinate, or null if none exists
+     */
+    public static Bundle getBundle(final BundleCoordinate bundleCoordinate) {
+        if (bundleCoordinate == null) {
+            throw new IllegalArgumentException("BundleCoordinate cannot be null");
+        }
+        return bundleCoordinateBundleLookup.get(bundleCoordinate);
+    }
+
+    /**
+     * Retrieves the bundle for the given class loader.
+     *
+     * @param classLoader the class loader to look up the bundle for
+     * @return the bundle for the given class loader
+     */
+    public static Bundle getBundle(final ClassLoader classLoader) {
+        if (classLoader == null) {
+            throw new IllegalArgumentException("ClassLoader cannot be null");
+        }
+        return classLoaderBundleLookup.get(classLoader);
+    }
+
+    public static Set<Class> getExtensions(final Class<?> definition) {
+        if (definition == null) {
+            throw new IllegalArgumentException("Class cannot be null");
+        }
+        final Set<Class> extensions = definitionMap.get(definition);
+        return (extensions == null) ? Collections.<Class>emptySet() : extensions;
+    }
+
+    public static ConfigurableComponent getTempComponent(final String classType, final BundleCoordinate bundleCoordinate) {
+        if (classType == null) {
+            throw new IllegalArgumentException("Class type cannot be null");
+        }
+
+        if (bundleCoordinate == null) {
+            throw new IllegalArgumentException("Bundle Coordinate cannot be null");
+        }
+
+        return tempComponentLookup.get(getClassBundleKey(classType, bundleCoordinate));
+    }
+
+    private static String getClassBundleKey(final String classType, final BundleCoordinate bundleCoordinate) {
+        return classType + "_" + bundleCoordinate.getCoordinate();
+    }
+
+    public static void logClassLoaderMapping() {
+        final StringBuilder builder = new StringBuilder();
+
+        builder.append("Extension Type Mapping to Bundle:");
+        for (final Map.Entry<Class, Set<Class>> entry : definitionMap.entrySet()) {
+            builder.append("\n\t=== ").append(entry.getKey().getSimpleName()).append(" Type ===");
+
+            for (final Class type : entry.getValue()) {
+                final List<Bundle> bundles = classNameBundleLookup.containsKey(type.getName())
+                        ? classNameBundleLookup.get(type.getName()) : Collections.emptyList();
+
+                builder.append("\n\t").append(type.getName());
+
+                for (final Bundle bundle : bundles) {
+                    final String coordinate = bundle.getBundleDetails().getCoordinate().getCoordinate();
+                    final String workingDir = bundle.getBundleDetails().getWorkingDirectory().getPath();
+                    builder.append("\n\t\t").append(coordinate).append(" || ").append(workingDir);
+                }
+            }
+
+            builder.append("\n\t=== End ").append(entry.getKey().getSimpleName()).append(" types ===");
+        }
+
+        logger.info(builder.toString());
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/556f309d/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/nar/InstanceClassLoader.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/nar/InstanceClassLoader.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/nar/InstanceClassLoader.java
new file mode 100644
index 0000000..d9e23fa
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/nar/InstanceClassLoader.java
@@ -0,0 +1,89 @@
+/*
+ * 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 org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.net.URL;
+import java.net.URLClassLoader;
+import java.util.Collections;
+import java.util.LinkedHashSet;
+import java.util.Set;
+
+/**
+ * Each processor, controller service, and reporting task will have an InstanceClassLoader.
+ *
+ * The InstanceClassLoader will either be an empty pass-through to the NARClassLoader, or will contain a
+ * copy of all the NAR's resources in the case of components that @RequireInstanceClassLoading.
+ */
+public class InstanceClassLoader extends URLClassLoader {
+
+    private static final Logger logger = LoggerFactory.getLogger(InstanceClassLoader.class);
+
+    private final String identifier;
+    private final String instanceType;
+
+    private final Set<URL> instanceUrls;
+    private final Set<URL> additionalResourceUrls;
+
+    /**
+     * @param identifier the id of the component this ClassLoader was created for
+     * @param instanceUrls the urls for the instance, will either be empty or a copy of the NARs urls
+     * @param additionalResourceUrls the urls that came from runtime properties of the component
+     * @param parent the parent ClassLoader
+     */
+    public InstanceClassLoader(final String identifier, final String type, final Set<URL> instanceUrls, final Set<URL> additionalResourceUrls, final ClassLoader parent) {
+        super(combineURLs(instanceUrls, additionalResourceUrls), parent);
+        this.identifier = identifier;
+        this.instanceType = type;
+        this.instanceUrls = Collections.unmodifiableSet(
+                instanceUrls == null ? Collections.emptySet() : new LinkedHashSet<>(instanceUrls));
+        this.additionalResourceUrls = Collections.unmodifiableSet(
+                additionalResourceUrls == null ? Collections.emptySet() : new LinkedHashSet<>(additionalResourceUrls));
+    }
+
+    private static URL[] combineURLs(final Set<URL> instanceUrls, final Set<URL> additionalResourceUrls) {
+        final Set<URL> allUrls = new LinkedHashSet<>();
+
+        if (instanceUrls != null) {
+            allUrls.addAll(instanceUrls);
+        }
+
+        if (additionalResourceUrls != null) {
+            allUrls.addAll(additionalResourceUrls);
+        }
+
+        return allUrls.toArray(new URL[allUrls.size()]);
+    }
+
+    public String getIdentifier() {
+        return identifier;
+    }
+
+    public String getInstanceType() {
+        return instanceType;
+    }
+
+    public Set<URL> getInstanceUrls() {
+        return instanceUrls;
+    }
+
+    public Set<URL> getAdditionalResourceUrls() {
+        return additionalResourceUrls;
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/556f309d/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/nar/NarCloseable.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/nar/NarCloseable.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/nar/NarCloseable.java
new file mode 100644
index 0000000..88d47ff
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/nar/NarCloseable.java
@@ -0,0 +1,112 @@
+/*
+ * 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 java.io.Closeable;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ *
+ */
+public class NarCloseable implements Closeable {
+
+    private static final Logger logger = LoggerFactory.getLogger(NarCloseable.class);
+
+    public static NarCloseable withNarLoader() {
+        final ClassLoader current = Thread.currentThread().getContextClassLoader();
+        Thread.currentThread().setContextClassLoader(NarThreadContextClassLoader.getInstance());
+        return new NarCloseable(current);
+    }
+
+    /**
+     * Sets the current thread context class loader to the specific appropriate class loader for the given
+     * component. If the component requires per-instance class loading then the class loader will be the
+     * specific class loader for instance with the given identifier, otherwise the class loader will be
+     * the NARClassLoader.
+     *
+     * @param componentClass the component class
+     * @param componentIdentifier the identifier of the component
+     * @return NarCloseable with the current thread context classloader jailed to the Nar
+     *              or instance class loader of the component
+     */
+    public static NarCloseable withComponentNarLoader(final Class componentClass, final String componentIdentifier) {
+        final ClassLoader current = Thread.currentThread().getContextClassLoader();
+
+        ClassLoader componentClassLoader = ExtensionManager.getInstanceClassLoader(componentIdentifier);
+        if (componentClassLoader == null) {
+            componentClassLoader = componentClass.getClassLoader();
+        }
+
+        Thread.currentThread().setContextClassLoader(componentClassLoader);
+        return new NarCloseable(current);
+    }
+
+    /**
+     * 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
+     *
+     * @return a NarCloseable
+     */
+    public static NarCloseable withFrameworkNar() {
+        final ClassLoader frameworkClassLoader;
+        try {
+            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.");
+            if (logger.isDebugEnabled()) {
+                logger.error("", e);
+            }
+
+            return new NarCloseable(null);
+        }
+
+        final ClassLoader current = Thread.currentThread().getContextClassLoader();
+        Thread.currentThread().setContextClassLoader(frameworkClassLoader);
+        return new NarCloseable(current);
+    }
+
+    private final ClassLoader toSet;
+
+    private NarCloseable(final ClassLoader toSet) {
+        this.toSet = toSet;
+    }
+
+    @Override
+    public void close() {
+        if (toSet != null) {
+            Thread.currentThread().setContextClassLoader(toSet);
+        }
+    }
+}


[6/7] nifi git commit: NIFI-3520 Refactoring instance class loading - Fixing FlowController to use appropriate class loader when instantiating processor - Updating ExtensionManager to leverage new flag in MANIFEST from NAR plugin - Adding ReloadComponent

Posted by mc...@apache.org.
http://git-wip-us.apache.org/repos/asf/nifi/blob/556f309d/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/fingerprint/FingerprintFactory.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/fingerprint/FingerprintFactory.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/fingerprint/FingerprintFactory.java
index 3bbe9e3..1db80fc 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/fingerprint/FingerprintFactory.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/fingerprint/FingerprintFactory.java
@@ -20,17 +20,13 @@ import org.apache.commons.lang3.StringUtils;
 import org.apache.nifi.bundle.BundleCoordinate;
 import org.apache.nifi.components.ConfigurableComponent;
 import org.apache.nifi.components.PropertyDescriptor;
-import org.apache.nifi.controller.ControllerService;
 import org.apache.nifi.controller.FlowController;
-import org.apache.nifi.controller.exception.ProcessorInstantiationException;
 import org.apache.nifi.controller.serialization.FlowFromDOMFactory;
-import org.apache.nifi.util.LoggingXmlParserErrorHandler;
 import org.apache.nifi.encrypt.StringEncryptor;
 import org.apache.nifi.nar.ExtensionManager;
-import org.apache.nifi.processor.Processor;
-import org.apache.nifi.reporting.ReportingTask;
 import org.apache.nifi.util.BundleUtils;
 import org.apache.nifi.util.DomUtils;
+import org.apache.nifi.util.LoggingXmlParserErrorHandler;
 import org.apache.nifi.web.api.dto.BundleDTO;
 import org.apache.nifi.web.api.dto.ControllerServiceDTO;
 import org.apache.nifi.web.api.dto.ReportingTaskDTO;
@@ -58,7 +54,6 @@ import java.util.List;
 import java.util.Map;
 import java.util.SortedMap;
 import java.util.TreeMap;
-import java.util.UUID;
 
 /**
  * <p>Creates a fingerprint of a flow.xml. The order of elements or attributes in the flow.xml does not influence the fingerprint generation.
@@ -232,7 +227,7 @@ public class FingerprintFactory {
             });
 
             for (final ControllerServiceDTO dto : serviceDtos) {
-                addControllerServiceFingerprint(builder, dto, controller);
+                addControllerServiceFingerprint(builder, dto);
             }
         }
 
@@ -262,7 +257,7 @@ public class FingerprintFactory {
             });
 
             for (final ReportingTaskDTO dto : reportingTaskDtos) {
-                addReportingTaskFingerprint(builder, dto, controller);
+                addReportingTaskFingerprint(builder, dto);
             }
         }
 
@@ -277,7 +272,7 @@ public class FingerprintFactory {
         final List<Element> processorElems = DomUtils.getChildElementsByTagName(processGroupElem, "processor");
         Collections.sort(processorElems, getIdsComparator());
         for (final Element processorElem : processorElems) {
-            addFlowFileProcessorFingerprint(builder, processorElem, controller);
+            addFlowFileProcessorFingerprint(builder, processorElem);
         }
 
         // input ports
@@ -332,7 +327,7 @@ public class FingerprintFactory {
         return builder;
     }
 
-    private StringBuilder addFlowFileProcessorFingerprint(final StringBuilder builder, final Element processorElem, final FlowController controller) throws FingerprintException {
+    private StringBuilder addFlowFileProcessorFingerprint(final StringBuilder builder, final Element processorElem) throws FingerprintException {
         // id
         appendFirstValue(builder, DomUtils.getChildNodesByTagName(processorElem, "id"));
         // class
@@ -346,24 +341,11 @@ public class FingerprintFactory {
         final BundleDTO bundle = FlowFromDOMFactory.getBundle(DomUtils.getChild(processorElem, "bundle"));
         addBundleFingerprint(builder, bundle);
 
-        // create an instance of the Processor so that we know the default property values
-        Processor processor = null;
-        try {
-            if (controller != null) {
-                final BundleCoordinate coordinate = getCoordinate(className, bundle);
-                processor = controller.createProcessor(className, UUID.randomUUID().toString(), coordinate, false).getProcessor();
-            }
-        } catch (ProcessorInstantiationException | IllegalStateException e) {
-            logger.warn("Unable to create Processor of type {} due to {}; its default properties will be fingerprinted instead of being ignored.", className, e.toString());
-            if (logger.isDebugEnabled()) {
-                logger.warn("", e);
-            }
-        } finally {
-            // The processor instance is only for fingerprinting so we can remove the InstanceClassLoader here
-            // since otherwise it will stick around in the map forever
-            if (processor != null) {
-                ExtensionManager.removeInstanceClassLoaderIfExists(processor.getIdentifier());
-            }
+        // get the temp instance of the Processor so that we know the default property values
+        final BundleCoordinate coordinate = getCoordinate(className, bundle);
+        final ConfigurableComponent configurableComponent = ExtensionManager.getTempComponent(className, coordinate);
+        if (configurableComponent == null) {
+            logger.warn("Unable to get Processor of type {}; its default properties will be fingerprinted instead of being ignored.", className);
         }
 
         // properties
@@ -372,7 +354,7 @@ public class FingerprintFactory {
         for (final Element propertyElem : sortedPropertyElems) {
             final String propName = DomUtils.getChildElementsByTagName(propertyElem, "name").get(0).getTextContent();
             String propValue = getFirstValue(DomUtils.getChildNodesByTagName(propertyElem, "value"), null);
-            addPropertyFingerprint(builder, processor, propName, propValue);
+            addPropertyFingerprint(builder, configurableComponent, propName, propValue);
         }
 
         final NodeList autoTerminateElems = DomUtils.getChildNodesByTagName(processorElem, "autoTerminatedRelationship");
@@ -571,7 +553,7 @@ public class FingerprintFactory {
         return builder;
     }
 
-    private void addControllerServiceFingerprint(final StringBuilder builder, final ControllerServiceDTO dto, final FlowController controller) {
+    private void addControllerServiceFingerprint(final StringBuilder builder, final ControllerServiceDTO dto) {
         builder.append(dto.getId());
         builder.append(dto.getType());
         builder.append(dto.getName());
@@ -582,21 +564,14 @@ public class FingerprintFactory {
         builder.append(dto.getAnnotationData());
         builder.append(dto.getState());
 
-        // create an instance of the ControllerService so that we know the default property values
-        ControllerService controllerService = null;
-        try {
-            if (controller != null) {
-                final BundleCoordinate coordinate = getCoordinate(dto.getType(), dto.getBundle());
-                controllerService = controller.createControllerService(dto.getType(), UUID.randomUUID().toString(), coordinate, false).getControllerServiceImplementation();
-            }
-        } catch (Exception e) {
-            logger.warn("Unable to create ControllerService of type {} due to {}; its default properties will be fingerprinted instead of being ignored.", dto.getType(), e.toString());
-            if (logger.isDebugEnabled()) {
-                logger.warn("", e);
-            }
+        // get the temp instance of the ControllerService so that we know the default property values
+        final BundleCoordinate coordinate = getCoordinate(dto.getType(), dto.getBundle());
+        final ConfigurableComponent configurableComponent = ExtensionManager.getTempComponent(dto.getType(), coordinate);
+        if (configurableComponent == null) {
+            logger.warn("Unable to get ControllerService of type {}; its default properties will be fingerprinted instead of being ignored.", dto.getType());
         }
 
-        addPropertiesFingerprint(builder, controllerService, dto.getProperties());
+        addPropertiesFingerprint(builder, configurableComponent, dto.getProperties());
     }
 
     private void addPropertiesFingerprint(final StringBuilder builder, final ConfigurableComponent component, final Map<String, String> properties) {
@@ -634,7 +609,7 @@ public class FingerprintFactory {
         return coordinate;
     }
 
-    private void addReportingTaskFingerprint(final StringBuilder builder, final ReportingTaskDTO dto, final FlowController controller) {
+    private void addReportingTaskFingerprint(final StringBuilder builder, final ReportingTaskDTO dto) {
         builder.append(dto.getId());
         builder.append(dto.getType());
         builder.append(dto.getName());
@@ -646,21 +621,14 @@ public class FingerprintFactory {
         builder.append(dto.getSchedulingStrategy());
         builder.append(dto.getAnnotationData());
 
-        // create an instance of the ReportingTask so that we know the default property values
-        ReportingTask reportingTask = null;
-        try {
-            if (controller != null) {
-                final BundleCoordinate coordinate = getCoordinate(dto.getType(), dto.getBundle());
-                reportingTask = controller.createReportingTask(dto.getType(), UUID.randomUUID().toString(), coordinate, false, false).getReportingTask();
-            }
-        } catch (Exception e) {
-            logger.warn("Unable to create ReportingTask of type {} due to {}; its default properties will be fingerprinted instead of being ignored.", dto.getType(), e.toString());
-            if (logger.isDebugEnabled()) {
-                logger.warn("", e);
-            }
+        // get the temp instance of the ReportingTask so that we know the default property values
+        final BundleCoordinate coordinate = getCoordinate(dto.getType(), dto.getBundle());
+        final ConfigurableComponent configurableComponent = ExtensionManager.getTempComponent(dto.getType(), coordinate);
+        if (configurableComponent == null) {
+            logger.warn("Unable to get ReportingTask of type {}; its default properties will be fingerprinted instead of being ignored.", dto.getType());
         }
 
-        addPropertiesFingerprint(builder, reportingTask, dto.getProperties());
+        addPropertiesFingerprint(builder, configurableComponent, dto.getProperties());
     }
 
     private Comparator<Element> getIdsComparator() {

http://git-wip-us.apache.org/repos/asf/nifi/blob/556f309d/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/groups/StandardProcessGroup.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/groups/StandardProcessGroup.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/groups/StandardProcessGroup.java
index 8d8fd19..452f3cc 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/groups/StandardProcessGroup.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/groups/StandardProcessGroup.java
@@ -767,7 +767,7 @@ public final class StandardProcessGroup implements ProcessGroup {
         } finally {
             if (removed) {
                 try {
-                    ExtensionManager.removeInstanceClassLoaderIfExists(id);
+                    ExtensionManager.removeInstanceClassLoader(id);
                 } catch (Throwable t) {
                 }
             }
@@ -1914,7 +1914,7 @@ public final class StandardProcessGroup implements ProcessGroup {
         } finally {
             if (removed) {
                 try {
-                    ExtensionManager.removeInstanceClassLoaderIfExists(service.getIdentifier());
+                    ExtensionManager.removeInstanceClassLoader(service.getIdentifier());
                 } catch (Throwable t) {
                 }
             }

http://git-wip-us.apache.org/repos/asf/nifi/blob/556f309d/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/groovy/org/apache/nifi/controller/StandardFlowSynchronizerSpec.groovy
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/groovy/org/apache/nifi/controller/StandardFlowSynchronizerSpec.groovy b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/groovy/org/apache/nifi/controller/StandardFlowSynchronizerSpec.groovy
index 738b25d..098814a 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/groovy/org/apache/nifi/controller/StandardFlowSynchronizerSpec.groovy
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/groovy/org/apache/nifi/controller/StandardFlowSynchronizerSpec.groovy
@@ -26,6 +26,7 @@ import org.apache.nifi.controller.queue.FlowFileQueue
 import org.apache.nifi.groups.ProcessGroup
 import org.apache.nifi.groups.RemoteProcessGroup
 import org.apache.nifi.nar.ExtensionManager
+import org.apache.nifi.nar.SystemBundle
 import org.apache.nifi.processor.Relationship
 import org.apache.nifi.reporting.BulletinRepository
 import org.apache.nifi.util.NiFiProperties
@@ -43,7 +44,7 @@ class StandardFlowSynchronizerSpec extends Specification {
         System.setProperty NiFiProperties.PROPERTIES_FILE_PATH, propFile
 
         def niFiProperties = NiFiProperties.createBasicNiFiProperties(null, null);
-        systemBundle = ExtensionManager.createSystemBundle(niFiProperties);
+        systemBundle = SystemBundle.create(niFiProperties);
         ExtensionManager.discoverExtensions(systemBundle, Collections.emptySet());
     }
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/556f309d/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/TestFlowController.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/TestFlowController.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/TestFlowController.java
index 557bd62..1b51be6 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/TestFlowController.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/TestFlowController.java
@@ -44,6 +44,8 @@ import org.apache.nifi.logging.LogLevel;
 import org.apache.nifi.logging.LogRepository;
 import org.apache.nifi.logging.LogRepositoryFactory;
 import org.apache.nifi.nar.ExtensionManager;
+import org.apache.nifi.nar.InstanceClassLoader;
+import org.apache.nifi.nar.SystemBundle;
 import org.apache.nifi.processor.Relationship;
 import org.apache.nifi.provenance.MockProvenanceRepository;
 import org.apache.nifi.registry.VariableRegistry;
@@ -67,6 +69,8 @@ import java.io.File;
 import java.io.FileInputStream;
 import java.io.IOException;
 import java.io.InputStream;
+import java.net.MalformedURLException;
+import java.net.URL;
 import java.nio.charset.StandardCharsets;
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -115,7 +119,7 @@ public class TestFlowController {
         encryptor = StringEncryptor.createEncryptor(nifiProperties);
 
         // use the system bundle
-        systemBundle = ExtensionManager.createSystemBundle(nifiProperties);
+        systemBundle = SystemBundle.create(nifiProperties);
         ExtensionManager.discoverExtensions(systemBundle, Collections.emptySet());
 
         User user1 = new User.Builder().identifier("user-id-1").identity("user-1").build();
@@ -465,7 +469,7 @@ public class TestFlowController {
     @Test
     public void testCreateMissingControllerService() throws ProcessorInstantiationException {
         final ControllerServiceNode serviceNode = controller.createControllerService("org.apache.nifi.NonExistingControllerService", "1234-Controller-Service",
-                systemBundle.getBundleDetails().getCoordinate(), false);
+                systemBundle.getBundleDetails().getCoordinate(), null, false);
         assertNotNull(serviceNode);
         assertEquals("org.apache.nifi.NonExistingControllerService", serviceNode.getCanonicalClassName());
         assertEquals("(Missing) NonExistingControllerService", serviceNode.getComponentType());
@@ -518,7 +522,7 @@ public class TestFlowController {
         ProcessGroup pg = controller.createProcessGroup("my-process-group");
         pg.setName("my-process-group");
         ControllerServiceNode cs = controller.createControllerService("org.apache.nifi.NonExistingControllerService", "my-controller-service",
-                systemBundle.getBundleDetails().getCoordinate(), false);
+                systemBundle.getBundleDetails().getCoordinate(), null, false);
         pg.addControllerService(cs);
         controller.getRootGroup().addProcessGroup(pg);
         controller.getRootGroup().removeProcessGroup(pg);
@@ -527,7 +531,7 @@ public class TestFlowController {
     }
 
     @Test
-    public void testChangeProcessorType() throws ProcessorInstantiationException {
+    public void testReloadProcessor() throws ProcessorInstantiationException {
         final String id = "1234-ScheduledProcessor" + System.currentTimeMillis();
         final BundleCoordinate coordinate = systemBundle.getBundleDetails().getCoordinate();
         final ProcessorNode processorNode = controller.createProcessor(DummyScheduledProcessor.class.getName(), id, coordinate);
@@ -548,7 +552,7 @@ public class TestFlowController {
         assertEquals(LogLevel.WARN, processorNode.getBulletinLevel());
 
         // now change the type of the processor from DummyScheduledProcessor to DummySettingsProcessor
-        controller.changeProcessorType(processorNode, DummySettingsProcessor.class.getName(), coordinate);
+        controller.reload(processorNode, DummySettingsProcessor.class.getName(), coordinate, Collections.emptySet());
 
         // ids and coordinate should stay the same
         assertEquals(id, processorNode.getIdentifier());
@@ -573,10 +577,42 @@ public class TestFlowController {
     }
 
     @Test
-    public void testChangeControllerServiceType() {
+    public void testReloadProcessorWithAdditionalResources() throws ProcessorInstantiationException, MalformedURLException {
+        final URL resource1 = new File("src/test/resources/TestClasspathResources/resource1.txt").toURI().toURL();
+        final URL resource2 = new File("src/test/resources/TestClasspathResources/resource2.txt").toURI().toURL();
+        final URL resource3 = new File("src/test/resources/TestClasspathResources/resource3.txt").toURI().toURL();
+        final Set<URL> additionalUrls = new LinkedHashSet<>(Arrays.asList(resource1, resource2, resource3));
+
+        final String id = "1234-ScheduledProcessor" + System.currentTimeMillis();
+        final BundleCoordinate coordinate = systemBundle.getBundleDetails().getCoordinate();
+        final ProcessorNode processorNode = controller.createProcessor(DummyScheduledProcessor.class.getName(), id, coordinate);
+        final String originalName = processorNode.getName();
+
+        // the instance class loader shouldn't have any of the resources yet
+        InstanceClassLoader instanceClassLoader = ExtensionManager.getInstanceClassLoader(id);
+        assertNotNull(instanceClassLoader);
+        assertFalse(containsResource(instanceClassLoader.getURLs(), resource1));
+        assertFalse(containsResource(instanceClassLoader.getURLs(), resource2));
+        assertFalse(containsResource(instanceClassLoader.getURLs(), resource3));
+        assertTrue(instanceClassLoader.getAdditionalResourceUrls().isEmpty());
+
+        // now change the type of the processor from DummyScheduledProcessor to DummySettingsProcessor
+        controller.reload(processorNode, DummySettingsProcessor.class.getName(), coordinate, additionalUrls);
+
+        // the instance class loader shouldn't have any of the resources yet
+        instanceClassLoader = ExtensionManager.getInstanceClassLoader(id);
+        assertNotNull(instanceClassLoader);
+        assertTrue(containsResource(instanceClassLoader.getURLs(), resource1));
+        assertTrue(containsResource(instanceClassLoader.getURLs(), resource2));
+        assertTrue(containsResource(instanceClassLoader.getURLs(), resource3));
+        assertEquals(3, instanceClassLoader.getAdditionalResourceUrls().size());
+    }
+
+    @Test
+    public void testReloadControllerService() {
         final String id = "ServiceA" + System.currentTimeMillis();
         final BundleCoordinate coordinate = systemBundle.getBundleDetails().getCoordinate();
-        final ControllerServiceNode controllerServiceNode = controller.createControllerService(ServiceA.class.getName(), id, coordinate, true);
+        final ControllerServiceNode controllerServiceNode = controller.createControllerService(ServiceA.class.getName(), id, coordinate, null, true);
         final String originalName = controllerServiceNode.getName();
 
         assertEquals(id, controllerServiceNode.getIdentifier());
@@ -586,7 +622,7 @@ public class TestFlowController {
         assertEquals(ServiceA.class.getSimpleName(), controllerServiceNode.getComponentType());
         assertEquals(ServiceA.class.getCanonicalName(), controllerServiceNode.getComponent().getClass().getCanonicalName());
 
-        controller.changeControllerServiceType(controllerServiceNode, ServiceB.class.getName(), coordinate);
+        controller.reload(controllerServiceNode, ServiceB.class.getName(), coordinate, Collections.emptySet());
 
         // ids and coordinate should stay the same
         assertEquals(id, controllerServiceNode.getIdentifier());
@@ -603,7 +639,38 @@ public class TestFlowController {
     }
 
     @Test
-    public void testChangeReportingTaskType() throws ReportingTaskInstantiationException {
+    public void testReloadControllerServiceWithAdditionalResources() throws MalformedURLException {
+        final URL resource1 = new File("src/test/resources/TestClasspathResources/resource1.txt").toURI().toURL();
+        final URL resource2 = new File("src/test/resources/TestClasspathResources/resource2.txt").toURI().toURL();
+        final URL resource3 = new File("src/test/resources/TestClasspathResources/resource3.txt").toURI().toURL();
+        final Set<URL> additionalUrls = new LinkedHashSet<>(Arrays.asList(resource1, resource2, resource3));
+
+        final String id = "ServiceA" + System.currentTimeMillis();
+        final BundleCoordinate coordinate = systemBundle.getBundleDetails().getCoordinate();
+        final ControllerServiceNode controllerServiceNode = controller.createControllerService(ServiceA.class.getName(), id, coordinate, null, true);
+        final String originalName = controllerServiceNode.getName();
+
+        // the instance class loader shouldn't have any of the resources yet
+        InstanceClassLoader instanceClassLoader = ExtensionManager.getInstanceClassLoader(id);
+        assertNotNull(instanceClassLoader);
+        assertFalse(containsResource(instanceClassLoader.getURLs(), resource1));
+        assertFalse(containsResource(instanceClassLoader.getURLs(), resource2));
+        assertFalse(containsResource(instanceClassLoader.getURLs(), resource3));
+        assertTrue(instanceClassLoader.getAdditionalResourceUrls().isEmpty());
+
+        controller.reload(controllerServiceNode, ServiceB.class.getName(), coordinate, additionalUrls);
+
+        // the instance class loader shouldn't have any of the resources yet
+        instanceClassLoader = ExtensionManager.getInstanceClassLoader(id);
+        assertNotNull(instanceClassLoader);
+        assertTrue(containsResource(instanceClassLoader.getURLs(), resource1));
+        assertTrue(containsResource(instanceClassLoader.getURLs(), resource2));
+        assertTrue(containsResource(instanceClassLoader.getURLs(), resource3));
+        assertEquals(3, instanceClassLoader.getAdditionalResourceUrls().size());
+    }
+
+    @Test
+    public void testReloadReportingTask() throws ReportingTaskInstantiationException {
         final String id = "ReportingTask" + System.currentTimeMillis();
         final BundleCoordinate coordinate = systemBundle.getBundleDetails().getCoordinate();
         final ReportingTaskNode node = controller.createReportingTask(DummyReportingTask.class.getName(), id, coordinate, true);
@@ -616,7 +683,7 @@ public class TestFlowController {
         assertEquals(DummyReportingTask.class.getSimpleName(), node.getComponentType());
         assertEquals(DummyReportingTask.class.getCanonicalName(), node.getComponent().getClass().getCanonicalName());
 
-        controller.changeReportingTaskType(node, DummyScheduledReportingTask.class.getName(), coordinate);
+        controller.reload(node, DummyScheduledReportingTask.class.getName(), coordinate, Collections.emptySet());
 
         // ids and coordinate should stay the same
         assertEquals(id, node.getIdentifier());
@@ -630,7 +697,45 @@ public class TestFlowController {
         assertEquals(DummyReportingTask.class.getCanonicalName(), node.getCanonicalClassName());
         assertEquals(DummyReportingTask.class.getSimpleName(), node.getComponentType());
         assertEquals(DummyScheduledReportingTask.class.getCanonicalName(), node.getComponent().getClass().getCanonicalName());
+    }
+
+    @Test
+    public void testReloadReportingTaskWithAdditionalResources() throws ReportingTaskInstantiationException, MalformedURLException {
+        final URL resource1 = new File("src/test/resources/TestClasspathResources/resource1.txt").toURI().toURL();
+        final URL resource2 = new File("src/test/resources/TestClasspathResources/resource2.txt").toURI().toURL();
+        final URL resource3 = new File("src/test/resources/TestClasspathResources/resource3.txt").toURI().toURL();
+        final Set<URL> additionalUrls = new LinkedHashSet<>(Arrays.asList(resource1, resource2, resource3));
 
+        final String id = "ReportingTask" + System.currentTimeMillis();
+        final BundleCoordinate coordinate = systemBundle.getBundleDetails().getCoordinate();
+        final ReportingTaskNode node = controller.createReportingTask(DummyReportingTask.class.getName(), id, coordinate, true);
+
+        // the instance class loader shouldn't have any of the resources yet
+        InstanceClassLoader instanceClassLoader = ExtensionManager.getInstanceClassLoader(id);
+        assertNotNull(instanceClassLoader);
+        assertFalse(containsResource(instanceClassLoader.getURLs(), resource1));
+        assertFalse(containsResource(instanceClassLoader.getURLs(), resource2));
+        assertFalse(containsResource(instanceClassLoader.getURLs(), resource3));
+        assertTrue(instanceClassLoader.getAdditionalResourceUrls().isEmpty());
+
+        controller.reload(node, DummyScheduledReportingTask.class.getName(), coordinate, additionalUrls);
+
+        // the instance class loader shouldn't have any of the resources yet
+        instanceClassLoader = ExtensionManager.getInstanceClassLoader(id);
+        assertNotNull(instanceClassLoader);
+        assertTrue(containsResource(instanceClassLoader.getURLs(), resource1));
+        assertTrue(containsResource(instanceClassLoader.getURLs(), resource2));
+        assertTrue(containsResource(instanceClassLoader.getURLs(), resource3));
+        assertEquals(3, instanceClassLoader.getAdditionalResourceUrls().size());
+    }
+
+    private boolean containsResource(URL[] resources, URL resourceToFind) {
+        for (URL resource : resources) {
+            if (resourceToFind.getPath().equals(resource.getPath())) {
+                return true;
+            }
+        }
+        return false;
     }
 
     @Test(expected = IllegalArgumentException.class)
@@ -744,7 +849,7 @@ public class TestFlowController {
     public void testInstantiateSnippetWhenControllerServiceMissingBundle() throws ProcessorInstantiationException {
         final String id = UUID.randomUUID().toString();
         final BundleCoordinate coordinate = systemBundle.getBundleDetails().getCoordinate();
-        final ControllerServiceNode controllerServiceNode = controller.createControllerService(ServiceA.class.getName(), id, coordinate, true);
+        final ControllerServiceNode controllerServiceNode = controller.createControllerService(ServiceA.class.getName(), id, coordinate, null, true);
 
         // create the controller service dto
         final ControllerServiceDTO csDto = new ControllerServiceDTO();
@@ -775,7 +880,7 @@ public class TestFlowController {
     public void testInstantiateSnippetWithControllerService() throws ProcessorInstantiationException {
         final String id = UUID.randomUUID().toString();
         final BundleCoordinate coordinate = systemBundle.getBundleDetails().getCoordinate();
-        final ControllerServiceNode controllerServiceNode = controller.createControllerService(ServiceA.class.getName(), id, coordinate, true);
+        final ControllerServiceNode controllerServiceNode = controller.createControllerService(ServiceA.class.getName(), id, coordinate, null, true);
 
         // create the controller service dto
         final ControllerServiceDTO csDto = new ControllerServiceDTO();

http://git-wip-us.apache.org/repos/asf/nifi/blob/556f309d/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/TestStandardProcessorNode.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/TestStandardProcessorNode.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/TestStandardProcessorNode.java
index a86b7b3..c248d25 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/TestStandardProcessorNode.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/TestStandardProcessorNode.java
@@ -25,12 +25,16 @@ import org.apache.nifi.bundle.BundleCoordinate;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.PropertyValue;
 import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.controller.exception.ControllerServiceInstantiationException;
+import org.apache.nifi.controller.exception.ProcessorInstantiationException;
+import org.apache.nifi.controller.reporting.ReportingTaskInstantiationException;
+import org.apache.nifi.controller.service.ControllerServiceNode;
 import org.apache.nifi.engine.FlowEngine;
 import org.apache.nifi.expression.ExpressionLanguageCompiler;
 import org.apache.nifi.logging.ComponentLog;
 import org.apache.nifi.nar.ExtensionManager;
-import org.apache.nifi.nar.InstanceClassLoader;
 import org.apache.nifi.nar.NarCloseable;
+import org.apache.nifi.nar.SystemBundle;
 import org.apache.nifi.processor.AbstractProcessor;
 import org.apache.nifi.processor.ProcessContext;
 import org.apache.nifi.processor.ProcessSession;
@@ -55,10 +59,10 @@ import org.mockito.Mockito;
 import java.io.File;
 import java.net.MalformedURLException;
 import java.net.URL;
-import java.net.URLClassLoader;
 import java.util.Arrays;
 import java.util.Collections;
 import java.util.HashMap;
+import java.util.LinkedHashSet;
 import java.util.Map;
 import java.util.Set;
 import java.util.UUID;
@@ -88,11 +92,12 @@ public class TestStandardProcessorNode {
         ProcessorInitializationContext initContext = new StandardProcessorInitializationContext(uuid, null, null, null, null);
         processor.initialize(initContext);
 
+        final ReloadComponent reloadComponent = Mockito.mock(ReloadComponent.class);
         final BundleCoordinate coordinate = Mockito.mock(BundleCoordinate.class);
 
         final LoggableComponent<Processor> loggableComponent = new LoggableComponent<>(processor, coordinate, null);
         final StandardProcessorNode procNode = new StandardProcessorNode(loggableComponent, uuid, createValidationContextFactory(), null, null,
-                NiFiProperties.createBasicNiFiProperties(null, null), VariableRegistry.EMPTY_REGISTRY);
+                NiFiProperties.createBasicNiFiProperties(null, null), VariableRegistry.EMPTY_REGISTRY, reloadComponent);
         final ScheduledExecutorService taskScheduler = new FlowEngine(2, "TestClasspathResources", true);
 
         final StandardProcessContext processContext = new StandardProcessContext(procNode, null, null, null, null);
@@ -122,8 +127,9 @@ public class TestStandardProcessorNode {
 
     @Test
     public void testDisabledValidationErrors() {
+        final MockReloadComponent reloadComponent = new MockReloadComponent();
         final ModifiesClasspathNoAnnotationProcessor processor = new ModifiesClasspathNoAnnotationProcessor();
-        final StandardProcessorNode procNode = createProcessorNode(processor);
+        final StandardProcessorNode procNode = createProcessorNode(processor, reloadComponent);
 
         // Set a property to an invalid value
         final Map<String, String> properties = new HashMap<>();
@@ -138,22 +144,19 @@ public class TestStandardProcessorNode {
 
     @Test
     public void testSinglePropertyDynamicallyModifiesClasspath() throws MalformedURLException {
+        final MockReloadComponent reloadComponent = new MockReloadComponent();
+
         final PropertyDescriptor classpathProp = new PropertyDescriptor.Builder().name("Classpath Resources")
                 .dynamicallyModifiesClasspath(true).addValidator(StandardValidators.NON_EMPTY_VALIDATOR).build();
         final ModifiesClasspathProcessor processor = new ModifiesClasspathProcessor(Arrays.asList(classpathProp));
-        final StandardProcessorNode procNode = createProcessorNode(processor);
+        final StandardProcessorNode procNode = createProcessorNode(processor, reloadComponent);
 
         try (final NarCloseable narCloseable = NarCloseable.withComponentNarLoader(procNode.getProcessor().getClass(), procNode.getIdentifier())){
-            // Should have an InstanceClassLoader here
-            final ClassLoader contextClassLoader = Thread.currentThread().getContextClassLoader();
-            assertTrue(contextClassLoader instanceof InstanceClassLoader);
-
-            final InstanceClassLoader instanceClassLoader = (InstanceClassLoader) contextClassLoader;
 
             // Should not have any of the test resources loaded at this point
             final URL[] testResources = getTestResources();
             for (URL testResource : testResources) {
-                if (containsResource(instanceClassLoader.getInstanceResources(), testResource)) {
+                if (containsResource(reloadComponent.getAdditionalUrls(), testResource)) {
                     fail("found resource that should not have been loaded");
                 }
             }
@@ -165,18 +168,22 @@ public class TestStandardProcessorNode {
 
             // Should have all of the resources loaded into the InstanceClassLoader now
             for (URL testResource : testResources) {
-                assertTrue(containsResource(instanceClassLoader.getInstanceResources(), testResource));
+                assertTrue(containsResource(reloadComponent.getAdditionalUrls(), testResource));
             }
 
+            assertEquals(ModifiesClasspathProcessor.class.getCanonicalName(), reloadComponent.getNewType());
+
             // Should pass validation
             assertTrue(procNode.isValid());
         } finally {
-            ExtensionManager.removeInstanceClassLoaderIfExists(procNode.getIdentifier());
+            ExtensionManager.removeInstanceClassLoader(procNode.getIdentifier());
         }
     }
 
     @Test
     public void testUpdateOtherPropertyDoesNotImpactClasspath() throws MalformedURLException {
+        final MockReloadComponent reloadComponent = new MockReloadComponent();
+
         final PropertyDescriptor classpathProp = new PropertyDescriptor.Builder().name("Classpath Resources")
                 .dynamicallyModifiesClasspath(true).addValidator(StandardValidators.NON_EMPTY_VALIDATOR).build();
 
@@ -184,19 +191,13 @@ public class TestStandardProcessorNode {
                 .addValidator(StandardValidators.NON_EMPTY_VALIDATOR).build();
 
         final ModifiesClasspathProcessor processor = new ModifiesClasspathProcessor(Arrays.asList(classpathProp, otherProp));
-        final StandardProcessorNode procNode = createProcessorNode(processor);
+        final StandardProcessorNode procNode = createProcessorNode(processor, reloadComponent);
 
         try (final NarCloseable narCloseable = NarCloseable.withComponentNarLoader(procNode.getProcessor().getClass(), procNode.getIdentifier())){
-            // Should have an InstanceClassLoader here
-            final ClassLoader contextClassLoader = Thread.currentThread().getContextClassLoader();
-            assertTrue(contextClassLoader instanceof InstanceClassLoader);
-
-            final InstanceClassLoader instanceClassLoader = (InstanceClassLoader) contextClassLoader;
-
             // Should not have any of the test resources loaded at this point
             final URL[] testResources = getTestResources();
             for (URL testResource : testResources) {
-                if (containsResource(instanceClassLoader.getInstanceResources(), testResource)) {
+                if (containsResource(reloadComponent.getAdditionalUrls(), testResource)) {
                     fail("found resource that should not have been loaded");
                 }
             }
@@ -208,7 +209,7 @@ public class TestStandardProcessorNode {
 
             // Should have all of the resources loaded into the InstanceClassLoader now
             for (URL testResource : testResources) {
-                assertTrue(containsResource(instanceClassLoader.getInstanceResources(), testResource));
+                assertTrue(containsResource(reloadComponent.getAdditionalUrls(), testResource));
             }
 
             // Should pass validation
@@ -221,7 +222,7 @@ public class TestStandardProcessorNode {
 
             // Should STILL have all of the resources loaded into the InstanceClassLoader now
             for (URL testResource : testResources) {
-                assertTrue(containsResource(instanceClassLoader.getInstanceResources(), testResource));
+                assertTrue(containsResource(reloadComponent.getAdditionalUrls(), testResource));
             }
 
             // Should STILL pass validation
@@ -233,38 +234,37 @@ public class TestStandardProcessorNode {
             procNode.setProperties(newClasspathProperties);
 
             // Should only have resource1 loaded now
-            assertTrue(containsResource(instanceClassLoader.getInstanceResources(), testResources[0]));
-            assertFalse(containsResource(instanceClassLoader.getInstanceResources(), testResources[1]));
-            assertFalse(containsResource(instanceClassLoader.getInstanceResources(), testResources[2]));
+            assertTrue(containsResource(reloadComponent.getAdditionalUrls(), testResources[0]));
+            assertFalse(containsResource(reloadComponent.getAdditionalUrls(), testResources[1]));
+            assertFalse(containsResource(reloadComponent.getAdditionalUrls(), testResources[2]));
+
+            assertEquals(ModifiesClasspathProcessor.class.getCanonicalName(), reloadComponent.getNewType());
 
             // Should STILL pass validation
             assertTrue(procNode.isValid());
         } finally {
-            ExtensionManager.removeInstanceClassLoaderIfExists(procNode.getIdentifier());
+            ExtensionManager.removeInstanceClassLoader(procNode.getIdentifier());
         }
     }
 
     @Test
     public void testMultiplePropertiesDynamicallyModifyClasspathWithExpressionLanguage() throws MalformedURLException {
+        final MockReloadComponent reloadComponent = new MockReloadComponent();
+
         final PropertyDescriptor classpathProp1 = new PropertyDescriptor.Builder().name("Classpath Resource 1")
                 .dynamicallyModifiesClasspath(true).addValidator(StandardValidators.NON_EMPTY_VALIDATOR).build();
         final PropertyDescriptor classpathProp2 = new PropertyDescriptor.Builder().name("Classpath Resource 2")
                 .dynamicallyModifiesClasspath(true).addValidator(StandardValidators.NON_EMPTY_VALIDATOR).build();
 
         final ModifiesClasspathProcessor processor = new ModifiesClasspathProcessor(Arrays.asList(classpathProp1, classpathProp2));
-        final StandardProcessorNode procNode = createProcessorNode(processor);
+        final StandardProcessorNode procNode = createProcessorNode(processor, reloadComponent);
 
         try (final NarCloseable narCloseable = NarCloseable.withComponentNarLoader(procNode.getProcessor().getClass(), procNode.getIdentifier())){
-            // Should have an InstanceClassLoader here
-            final ClassLoader contextClassLoader = Thread.currentThread().getContextClassLoader();
-            assertTrue(contextClassLoader instanceof InstanceClassLoader);
-
-            final InstanceClassLoader instanceClassLoader = (InstanceClassLoader) contextClassLoader;
 
             // Should not have any of the test resources loaded at this point
             final URL[] testResources = getTestResources();
             for (URL testResource : testResources) {
-                if (containsResource(instanceClassLoader.getInstanceResources(), testResource)) {
+                if (containsResource(reloadComponent.getAdditionalUrls(), testResource)) {
                     fail("found resource that should not have been loaded");
                 }
             }
@@ -279,38 +279,37 @@ public class TestStandardProcessorNode {
             procNode.setProperties(properties);
 
             // Should have resources 1 and 3 loaded into the InstanceClassLoader now
-            assertTrue(containsResource(instanceClassLoader.getInstanceResources(), testResources[0]));
-            assertTrue(containsResource(instanceClassLoader.getInstanceResources(), testResources[2]));
-            assertFalse(containsResource(instanceClassLoader.getInstanceResources(), testResources[1]));
+            assertTrue(containsResource(reloadComponent.getAdditionalUrls(), testResources[0]));
+            assertTrue(containsResource(reloadComponent.getAdditionalUrls(), testResources[2]));
+            assertFalse(containsResource(reloadComponent.getAdditionalUrls(), testResources[1]));
+
+            assertEquals(ModifiesClasspathProcessor.class.getCanonicalName(), reloadComponent.getNewType());
 
             // Should pass validation
             assertTrue(procNode.isValid());
         } finally {
-            ExtensionManager.removeInstanceClassLoaderIfExists(procNode.getIdentifier());
+            ExtensionManager.removeInstanceClassLoader(procNode.getIdentifier());
         }
     }
 
     @Test
     public void testSomeNonExistentPropertiesDynamicallyModifyClasspath() throws MalformedURLException {
+        final MockReloadComponent reloadComponent = new MockReloadComponent();
+
         final PropertyDescriptor classpathProp1 = new PropertyDescriptor.Builder().name("Classpath Resource 1")
                 .dynamicallyModifiesClasspath(true).addValidator(StandardValidators.NON_EMPTY_VALIDATOR).build();
         final PropertyDescriptor classpathProp2 = new PropertyDescriptor.Builder().name("Classpath Resource 2")
                 .dynamicallyModifiesClasspath(true).addValidator(StandardValidators.NON_EMPTY_VALIDATOR).build();
 
         final ModifiesClasspathProcessor processor = new ModifiesClasspathProcessor(Arrays.asList(classpathProp1, classpathProp2));
-        final StandardProcessorNode procNode = createProcessorNode(processor);
+        final StandardProcessorNode procNode = createProcessorNode(processor, reloadComponent);
 
         try (final NarCloseable narCloseable = NarCloseable.withComponentNarLoader(procNode.getProcessor().getClass(), procNode.getIdentifier())){
-            // Should have an InstanceClassLoader here
-            final ClassLoader contextClassLoader = Thread.currentThread().getContextClassLoader();
-            assertTrue(contextClassLoader instanceof InstanceClassLoader);
-
-            final InstanceClassLoader instanceClassLoader = (InstanceClassLoader) contextClassLoader;
 
             // Should not have any of the test resources loaded at this point
             final URL[] testResources = getTestResources();
             for (URL testResource : testResources) {
-                if (containsResource(instanceClassLoader.getInstanceResources(), testResource)) {
+                if (containsResource(reloadComponent.getAdditionalUrls(), testResource)) {
                     fail("found resource that should not have been loaded");
                 }
             }
@@ -322,53 +321,52 @@ public class TestStandardProcessorNode {
             procNode.setProperties(properties);
 
             // Should have resources 1 and 3 loaded into the InstanceClassLoader now
-            assertTrue(containsResource(instanceClassLoader.getInstanceResources(), testResources[0]));
-            assertFalse(containsResource(instanceClassLoader.getInstanceResources(), testResources[1]));
-            assertFalse(containsResource(instanceClassLoader.getInstanceResources(), testResources[2]));
+            assertTrue(containsResource(reloadComponent.getAdditionalUrls(), testResources[0]));
+            assertFalse(containsResource(reloadComponent.getAdditionalUrls(), testResources[1]));
+            assertFalse(containsResource(reloadComponent.getAdditionalUrls(), testResources[2]));
+
+            assertEquals(ModifiesClasspathProcessor.class.getCanonicalName(), reloadComponent.getNewType());
 
             // Should pass validation
             assertTrue(procNode.isValid());
         } finally {
-            ExtensionManager.removeInstanceClassLoaderIfExists(procNode.getIdentifier());
+            ExtensionManager.removeInstanceClassLoader(procNode.getIdentifier());
         }
     }
 
     @Test
     public void testPropertyModifiesClasspathWhenProcessorMissingAnnotation() throws MalformedURLException {
+        final MockReloadComponent reloadComponent = new MockReloadComponent();
         final ModifiesClasspathNoAnnotationProcessor processor = new ModifiesClasspathNoAnnotationProcessor();
-        final StandardProcessorNode procNode = createProcessorNode(processor);
+        final StandardProcessorNode procNode = createProcessorNode(processor, reloadComponent);
 
         try (final NarCloseable narCloseable = NarCloseable.withComponentNarLoader(procNode.getProcessor().getClass(), procNode.getIdentifier())){
-            // Can't validate the ClassLoader here b/c the class is missing the annotation
 
-            // Simulate setting the properties pointing to two of the resources
             final Map<String, String> properties = new HashMap<>();
             properties.put(ModifiesClasspathNoAnnotationProcessor.CLASSPATH_RESOURCE.getName(),
                     "src/test/resources/TestClasspathResources/resource1.txt");
             procNode.setProperties(properties);
 
-            // Should not have loaded any of the resources
-            final ClassLoader classLoader = Thread.currentThread().getContextClassLoader();
-            assertTrue(classLoader instanceof URLClassLoader);
-
             final URL[] testResources = getTestResources();
-            final URLClassLoader urlClassLoader = (URLClassLoader) classLoader;
-            assertFalse(containsResource(urlClassLoader.getURLs(), testResources[0]));
-            assertFalse(containsResource(urlClassLoader.getURLs(), testResources[1]));
-            assertFalse(containsResource(urlClassLoader.getURLs(), testResources[2]));
+            assertTrue(containsResource(reloadComponent.getAdditionalUrls(), testResources[0]));
+            assertFalse(containsResource(reloadComponent.getAdditionalUrls(), testResources[1]));
+            assertFalse(containsResource(reloadComponent.getAdditionalUrls(), testResources[2]));
+
+            assertEquals(ModifiesClasspathNoAnnotationProcessor.class.getCanonicalName(), reloadComponent.getNewType());
 
             // Should pass validation
             assertTrue(procNode.isValid());
 
         } finally {
-            ExtensionManager.removeInstanceClassLoaderIfExists(procNode.getIdentifier());
+            ExtensionManager.removeInstanceClassLoader(procNode.getIdentifier());
         }
     }
 
     @Test
     public void testVerifyCanUpdateBundle() {
+        final ReloadComponent reloadComponent = new MockReloadComponent();
         final ModifiesClasspathNoAnnotationProcessor processor = new ModifiesClasspathNoAnnotationProcessor();
-        final StandardProcessorNode procNode = createProcessorNode(processor);
+        final StandardProcessorNode procNode = createProcessorNode(processor, reloadComponent);
         final BundleCoordinate existingCoordinate = procNode.getBundleCoordinate();
 
         // should be allowed to update when the bundle is the same
@@ -400,30 +398,68 @@ public class TestStandardProcessorNode {
         }
     }
 
-    @Test
-    public void testValidateControllerServiceApiRequired() {
-
-    }
-
-    private StandardProcessorNode createProcessorNode(Processor processor) {
+    private StandardProcessorNode createProcessorNode(final Processor processor, final ReloadComponent reloadComponent) {
         final String uuid = UUID.randomUUID().toString();
         final ValidationContextFactory validationContextFactory = createValidationContextFactory();
         final NiFiProperties niFiProperties = NiFiProperties.createBasicNiFiProperties("src/test/resources/conf/nifi.properties", null);
         final ProcessScheduler processScheduler = Mockito.mock(ProcessScheduler.class);
         final ComponentLog componentLog = Mockito.mock(ComponentLog.class);
 
-        final Bundle systemBundle = ExtensionManager.createSystemBundle(niFiProperties);
+        final Bundle systemBundle = SystemBundle.create(niFiProperties);
         ExtensionManager.discoverExtensions(systemBundle, Collections.emptySet());
-        ExtensionManager.createInstanceClassLoader(processor.getClass().getName(), uuid, systemBundle);
+        ExtensionManager.createInstanceClassLoader(processor.getClass().getName(), uuid, systemBundle, null);
 
         ProcessorInitializationContext initContext = new StandardProcessorInitializationContext(uuid, componentLog, null, null, null);
         processor.initialize(initContext);
 
         final LoggableComponent<Processor> loggableComponent = new LoggableComponent<>(processor, systemBundle.getBundleDetails().getCoordinate(), componentLog);
-        return new StandardProcessorNode(loggableComponent, uuid, validationContextFactory, processScheduler, null, niFiProperties, variableRegistry);
+        return new StandardProcessorNode(loggableComponent, uuid, validationContextFactory, processScheduler, null, niFiProperties, variableRegistry, reloadComponent);
+    }
+
+    private static class MockReloadComponent implements ReloadComponent {
+
+        private String newType;
+        private BundleCoordinate bundleCoordinate;
+        private final Set<URL> additionalUrls = new LinkedHashSet<>();
+
+        public Set<URL> getAdditionalUrls() {
+            return this.additionalUrls;
+        }
+
+        public String getNewType() {
+            return newType;
+        }
+
+        public BundleCoordinate getBundleCoordinate() {
+            return bundleCoordinate;
+        }
+
+        @Override
+        public void reload(ProcessorNode existingNode, String newType, BundleCoordinate bundleCoordinate, Set<URL> additionalUrls) throws ProcessorInstantiationException {
+            reload(newType, bundleCoordinate, additionalUrls);
+        }
+
+        @Override
+        public void reload(ControllerServiceNode existingNode, String newType, BundleCoordinate bundleCoordinate, Set<URL> additionalUrls) throws ControllerServiceInstantiationException {
+            reload(newType, bundleCoordinate, additionalUrls);
+        }
+
+        @Override
+        public void reload(ReportingTaskNode existingNode, String newType, BundleCoordinate bundleCoordinate, Set<URL> additionalUrls) throws ReportingTaskInstantiationException {
+            reload(newType, bundleCoordinate, additionalUrls);
+        }
+
+        private void reload(String newType, BundleCoordinate bundleCoordinate, Set<URL> additionalUrls) {
+            this.newType = newType;
+            this.bundleCoordinate = bundleCoordinate;
+            this.additionalUrls.clear();
+            if (additionalUrls != null) {
+                this.additionalUrls.addAll(additionalUrls);
+            }
+        }
     }
 
-    private boolean containsResource(URL[] resources, URL resourceToFind) {
+    private boolean containsResource(Set<URL> resources, URL resourceToFind) {
         for (URL resource : resources) {
             if (resourceToFind.getPath().equals(resource.getPath())) {
                 return true;

http://git-wip-us.apache.org/repos/asf/nifi/blob/556f309d/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/scheduling/TestProcessorLifecycle.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/scheduling/TestProcessorLifecycle.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/scheduling/TestProcessorLifecycle.java
index 5c8d447..9533751 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/scheduling/TestProcessorLifecycle.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/scheduling/TestProcessorLifecycle.java
@@ -39,6 +39,7 @@ import org.apache.nifi.controller.service.ControllerServiceNode;
 import org.apache.nifi.events.VolatileBulletinRepository;
 import org.apache.nifi.groups.ProcessGroup;
 import org.apache.nifi.nar.ExtensionManager;
+import org.apache.nifi.nar.SystemBundle;
 import org.apache.nifi.processor.AbstractProcessor;
 import org.apache.nifi.processor.ProcessContext;
 import org.apache.nifi.processor.ProcessSession;
@@ -542,7 +543,7 @@ public class TestProcessorLifecycle {
         this.setControllerRootGroup(fc, testGroup);
 
         ControllerServiceNode testServiceNode = fc.createControllerService(TestService.class.getName(), "serv",
-                fcsb.getSystemBundle().getBundleDetails().getCoordinate(), true);
+                fcsb.getSystemBundle().getBundleDetails().getCoordinate(), null, true);
         ProcessorNode testProcNode = fc.createProcessor(TestProcessor.class.getName(), UUID.randomUUID().toString(),
                 fcsb.getSystemBundle().getBundleDetails().getCoordinate());
 
@@ -569,7 +570,7 @@ public class TestProcessorLifecycle {
         this.setControllerRootGroup(fc, testGroup);
 
         ControllerServiceNode testServiceNode = fc.createControllerService(TestService.class.getName(), "foo",
-                fcsb.getSystemBundle().getBundleDetails().getCoordinate(), true);
+                fcsb.getSystemBundle().getBundleDetails().getCoordinate(), null, true);
         testGroup.addControllerService(testServiceNode);
 
         ProcessorNode testProcNode = fc.createProcessor(TestProcessor.class.getName(), UUID.randomUUID().toString(),
@@ -731,7 +732,7 @@ public class TestProcessorLifecycle {
         }
         final NiFiProperties nifiProperties = NiFiProperties.createBasicNiFiProperties(null, addProps);
 
-        final Bundle systemBundle = ExtensionManager.createSystemBundle(nifiProperties);
+        final Bundle systemBundle = SystemBundle.create(nifiProperties);
         ExtensionManager.discoverExtensions(systemBundle, Collections.emptySet());
 
         final FlowController flowController = FlowController.createStandaloneInstance(mock(FlowFileEventRepository.class), nifiProperties,

http://git-wip-us.apache.org/repos/asf/nifi/blob/556f309d/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/scheduling/TestStandardProcessScheduler.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/scheduling/TestStandardProcessScheduler.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/scheduling/TestStandardProcessScheduler.java
index b69701e..dbbd614 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/scheduling/TestStandardProcessScheduler.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/scheduling/TestStandardProcessScheduler.java
@@ -28,6 +28,7 @@ import org.apache.nifi.controller.FlowController;
 import org.apache.nifi.controller.LoggableComponent;
 import org.apache.nifi.controller.ProcessScheduler;
 import org.apache.nifi.controller.ProcessorNode;
+import org.apache.nifi.controller.ReloadComponent;
 import org.apache.nifi.controller.ReportingTaskNode;
 import org.apache.nifi.controller.StandardProcessorNode;
 import org.apache.nifi.controller.ValidationContextFactory;
@@ -43,6 +44,7 @@ import org.apache.nifi.controller.service.mock.MockProcessGroup;
 import org.apache.nifi.groups.ProcessGroup;
 import org.apache.nifi.logging.ComponentLog;
 import org.apache.nifi.nar.ExtensionManager;
+import org.apache.nifi.nar.SystemBundle;
 import org.apache.nifi.processor.AbstractProcessor;
 import org.apache.nifi.processor.ProcessContext;
 import org.apache.nifi.processor.ProcessSession;
@@ -98,7 +100,7 @@ public class TestStandardProcessScheduler {
         this.nifiProperties = NiFiProperties.createBasicNiFiProperties(null, null);
 
         // load the system bundle
-        systemBundle = ExtensionManager.createSystemBundle(nifiProperties);
+        systemBundle = SystemBundle.create(nifiProperties);
         ExtensionManager.discoverExtensions(systemBundle, Collections.emptySet());
 
         scheduler = new StandardProcessScheduler(Mockito.mock(ControllerServiceProvider.class), null, stateMgrProvider, variableRegistry, nifiProperties);
@@ -111,8 +113,9 @@ public class TestStandardProcessScheduler {
 
         final ValidationContextFactory validationContextFactory = new StandardValidationContextFactory(null, variableRegistry);
         final ComponentLog logger = Mockito.mock(ComponentLog.class);
+        final ReloadComponent reloadComponent = Mockito.mock(ReloadComponent.class);
         final LoggableComponent<ReportingTask> loggableComponent = new LoggableComponent<>(reportingTask, systemBundle.getBundleDetails().getCoordinate(), logger);
-        taskNode = new StandardReportingTaskNode(loggableComponent, UUID.randomUUID().toString(), null, scheduler, validationContextFactory, variableRegistry);
+        taskNode = new StandardReportingTaskNode(loggableComponent, UUID.randomUUID().toString(), null, scheduler, validationContextFactory, variableRegistry, reloadComponent);
 
         controller = Mockito.mock(FlowController.class);
         rootGroup = new MockProcessGroup();
@@ -150,16 +153,18 @@ public class TestStandardProcessScheduler {
         final Processor proc = new ServiceReferencingProcessor();
         proc.initialize(new StandardProcessorInitializationContext(uuid, null, null, null, null));
 
+        final ReloadComponent reloadComponent = Mockito.mock(ReloadComponent.class);
+
         final StandardControllerServiceProvider serviceProvider =
                 new StandardControllerServiceProvider(controller, scheduler, null, Mockito.mock(StateManagerProvider.class), variableRegistry, nifiProperties);
         final ControllerServiceNode service = serviceProvider.createControllerService(NoStartServiceImpl.class.getName(), "service",
-                systemBundle.getBundleDetails().getCoordinate(), true);
+                systemBundle.getBundleDetails().getCoordinate(), null, true);
         rootGroup.addControllerService(service);
 
         final LoggableComponent<Processor> loggableComponent = new LoggableComponent<>(proc, systemBundle.getBundleDetails().getCoordinate(), null);
         final ProcessorNode procNode = new StandardProcessorNode(loggableComponent, uuid,
                 new StandardValidationContextFactory(serviceProvider, variableRegistry),
-                scheduler, serviceProvider, nifiProperties, VariableRegistry.EMPTY_REGISTRY);
+                scheduler, serviceProvider, nifiProperties, VariableRegistry.EMPTY_REGISTRY, reloadComponent);
         rootGroup.addProcessor(procNode);
 
         Map<String,String> procProps = new HashMap<>();
@@ -233,7 +238,7 @@ public class TestStandardProcessScheduler {
         final ProcessScheduler scheduler = createScheduler();
         final StandardControllerServiceProvider provider = new StandardControllerServiceProvider(controller, scheduler, null, stateMgrProvider, variableRegistry, nifiProperties);
         final ControllerServiceNode serviceNode = provider.createControllerService(SimpleTestService.class.getName(),
-                "1", systemBundle.getBundleDetails().getCoordinate(), false);
+                "1", systemBundle.getBundleDetails().getCoordinate(), null, false);
         assertFalse(serviceNode.isActive());
         final SimpleTestService ts = (SimpleTestService) serviceNode.getControllerServiceImplementation();
         final ExecutorService executor = Executors.newCachedThreadPool();
@@ -272,7 +277,7 @@ public class TestStandardProcessScheduler {
         final ProcessScheduler scheduler = createScheduler();
         final StandardControllerServiceProvider provider = new StandardControllerServiceProvider(controller, scheduler, null, stateMgrProvider, variableRegistry, nifiProperties);
         final ControllerServiceNode serviceNode = provider.createControllerService(SimpleTestService.class.getName(),
-                "1", systemBundle.getBundleDetails().getCoordinate(), false);
+                "1", systemBundle.getBundleDetails().getCoordinate(), null, false);
         final SimpleTestService ts = (SimpleTestService) serviceNode.getControllerServiceImplementation();
         final ExecutorService executor = Executors.newCachedThreadPool();
 
@@ -310,7 +315,7 @@ public class TestStandardProcessScheduler {
         final ProcessScheduler scheduler = createScheduler();
         final StandardControllerServiceProvider provider = new StandardControllerServiceProvider(controller, scheduler, null, stateMgrProvider, variableRegistry, nifiProperties);
         final ControllerServiceNode serviceNode = provider.createControllerService(SimpleTestService.class.getName(),
-                "1", systemBundle.getBundleDetails().getCoordinate(), false);
+                "1", systemBundle.getBundleDetails().getCoordinate(), null, false);
         final SimpleTestService ts = (SimpleTestService) serviceNode.getControllerServiceImplementation();
         scheduler.enableControllerService(serviceNode);
         assertTrue(serviceNode.isActive());
@@ -344,7 +349,7 @@ public class TestStandardProcessScheduler {
         final ProcessScheduler scheduler = createScheduler();
         final StandardControllerServiceProvider provider = new StandardControllerServiceProvider(controller, scheduler, null, stateMgrProvider, variableRegistry, nifiProperties);
         final ControllerServiceNode serviceNode = provider.createControllerService(FailingService.class.getName(),
-                "1", systemBundle.getBundleDetails().getCoordinate(), false);
+                "1", systemBundle.getBundleDetails().getCoordinate(), null, false);
         scheduler.enableControllerService(serviceNode);
         Thread.sleep(1000);
         scheduler.shutdown();
@@ -378,7 +383,7 @@ public class TestStandardProcessScheduler {
         final ExecutorService executor = Executors.newCachedThreadPool();
         for (int i = 0; i < 200; i++) {
             final ControllerServiceNode serviceNode = provider.createControllerService(RandomShortDelayEnablingService.class.getName(), "1",
-                    systemBundle.getBundleDetails().getCoordinate(), false);
+                    systemBundle.getBundleDetails().getCoordinate(), null, false);
 
             executor.execute(new Runnable() {
                 @Override
@@ -419,7 +424,7 @@ public class TestStandardProcessScheduler {
         final ProcessScheduler scheduler = createScheduler();
         final StandardControllerServiceProvider provider = new StandardControllerServiceProvider(controller, scheduler, null, stateMgrProvider, variableRegistry, nifiProperties);
         final ControllerServiceNode serviceNode = provider.createControllerService(LongEnablingService.class.getName(),
-                "1", systemBundle.getBundleDetails().getCoordinate(), false);
+                "1", systemBundle.getBundleDetails().getCoordinate(), null, false);
         final LongEnablingService ts = (LongEnablingService) serviceNode.getControllerServiceImplementation();
         ts.setLimit(Long.MAX_VALUE);
         scheduler.enableControllerService(serviceNode);
@@ -445,7 +450,7 @@ public class TestStandardProcessScheduler {
         final ProcessScheduler scheduler = createScheduler();
         final StandardControllerServiceProvider provider = new StandardControllerServiceProvider(controller, scheduler, null, stateMgrProvider, variableRegistry, nifiProperties);
         final ControllerServiceNode serviceNode = provider.createControllerService(LongEnablingService.class.getName(),
-                "1", systemBundle.getBundleDetails().getCoordinate(), false);
+                "1", systemBundle.getBundleDetails().getCoordinate(), null, false);
         final LongEnablingService ts = (LongEnablingService) serviceNode.getControllerServiceImplementation();
         ts.setLimit(3000);
         scheduler.enableControllerService(serviceNode);

http://git-wip-us.apache.org/repos/asf/nifi/blob/556f309d/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/StandardControllerServiceProviderTest.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/StandardControllerServiceProviderTest.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/StandardControllerServiceProviderTest.java
index 15c35d9..4a97b8a 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/StandardControllerServiceProviderTest.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/StandardControllerServiceProviderTest.java
@@ -22,6 +22,7 @@ import org.apache.nifi.components.state.StateManagerProvider;
 import org.apache.nifi.controller.ControllerService;
 import org.apache.nifi.nar.ExtensionManager;
 import org.apache.nifi.nar.NarClassLoaders;
+import org.apache.nifi.nar.SystemBundle;
 import org.apache.nifi.registry.VariableRegistry;
 import org.apache.nifi.reporting.InitializationException;
 import org.apache.nifi.util.FileBasedVariableRegistry;
@@ -48,7 +49,7 @@ public class StandardControllerServiceProviderTest {
         NarClassLoaders.getInstance().init(nifiProperties.getFrameworkWorkingDirectory(), nifiProperties.getExtensionsWorkingDirectory());
 
         // load the system bundle
-        systemBundle = ExtensionManager.createSystemBundle(nifiProperties);
+        systemBundle = SystemBundle.create(nifiProperties);
         ExtensionManager.discoverExtensions(systemBundle, NarClassLoaders.getInstance().getBundles());
 
         variableRegistry = new FileBasedVariableRegistry(nifiProperties.getVariableRegistryPropertiesPaths());
@@ -80,7 +81,7 @@ public class StandardControllerServiceProviderTest {
             public void onComponentRemoved(String componentId) {
             }
         }, variableRegistry, nifiProperties);
-        ControllerServiceNode node = provider.createControllerService(clazz, id, systemBundle.getBundleDetails().getCoordinate(), true);
+        ControllerServiceNode node = provider.createControllerService(clazz, id, systemBundle.getBundleDetails().getCoordinate(), null, true);
         proxied = node.getProxiedControllerService();
         implementation = node.getControllerServiceImplementation();
     }

http://git-wip-us.apache.org/repos/asf/nifi/blob/556f309d/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/TestStandardControllerServiceProvider.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/TestStandardControllerServiceProvider.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/TestStandardControllerServiceProvider.java
index 40d9357..3a28cb0 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/TestStandardControllerServiceProvider.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/TestStandardControllerServiceProvider.java
@@ -24,6 +24,7 @@ import org.apache.nifi.controller.FlowController;
 import org.apache.nifi.controller.LoggableComponent;
 import org.apache.nifi.controller.ProcessScheduler;
 import org.apache.nifi.controller.ProcessorNode;
+import org.apache.nifi.controller.ReloadComponent;
 import org.apache.nifi.controller.ScheduledState;
 import org.apache.nifi.controller.StandardProcessorNode;
 import org.apache.nifi.controller.scheduling.StandardProcessScheduler;
@@ -35,6 +36,7 @@ import org.apache.nifi.controller.service.mock.ServiceC;
 import org.apache.nifi.groups.ProcessGroup;
 import org.apache.nifi.groups.StandardProcessGroup;
 import org.apache.nifi.nar.ExtensionManager;
+import org.apache.nifi.nar.SystemBundle;
 import org.apache.nifi.processor.Processor;
 import org.apache.nifi.processor.StandardValidationContextFactory;
 import org.apache.nifi.registry.VariableRegistry;
@@ -93,7 +95,7 @@ public class TestStandardControllerServiceProvider {
         niFiProperties = NiFiProperties.createBasicNiFiProperties(null, null);
 
         // load the system bundle
-        systemBundle = ExtensionManager.createSystemBundle(niFiProperties);
+        systemBundle = SystemBundle.create(niFiProperties);
         ExtensionManager.discoverExtensions(systemBundle, Collections.emptySet());
     }
 
@@ -118,7 +120,7 @@ public class TestStandardControllerServiceProvider {
                 new StandardControllerServiceProvider(controller, scheduler, null, stateManagerProvider, variableRegistry, niFiProperties);
 
         final ControllerServiceNode serviceNode = provider.createControllerService(ServiceB.class.getName(), "B",
-                systemBundle.getBundleDetails().getCoordinate(), false);
+                systemBundle.getBundleDetails().getCoordinate(), null,false);
         provider.enableControllerService(serviceNode);
         provider.disableControllerService(serviceNode);
     }
@@ -134,9 +136,9 @@ public class TestStandardControllerServiceProvider {
                 new StandardControllerServiceProvider(controller, scheduler, null, stateManagerProvider, variableRegistry, niFiProperties);
 
         final ControllerServiceNode serviceNodeB = provider.createControllerService(ServiceB.class.getName(), "B",
-                systemBundle.getBundleDetails().getCoordinate(), false);
+                systemBundle.getBundleDetails().getCoordinate(), null, false);
         final ControllerServiceNode serviceNodeA = provider.createControllerService(ServiceA.class.getName(), "A",
-                systemBundle.getBundleDetails().getCoordinate(), false);
+                systemBundle.getBundleDetails().getCoordinate(), null, false);
         group.addControllerService(serviceNodeA);
         group.addControllerService(serviceNodeB);
 
@@ -208,13 +210,13 @@ public class TestStandardControllerServiceProvider {
         // we enable C and B, even if we attempt to enable C before B... i.e., if we try to enable C, we cannot do so
         // until B is first enabled so ensure that we enable B first.
         final ControllerServiceNode serviceNode1 = provider.createControllerService(ServiceA.class.getName(), "1",
-                systemBundle.getBundleDetails().getCoordinate(), false);
+                systemBundle.getBundleDetails().getCoordinate(), null, false);
         final ControllerServiceNode serviceNode2 = provider.createControllerService(ServiceA.class.getName(), "2",
-                systemBundle.getBundleDetails().getCoordinate(), false);
+                systemBundle.getBundleDetails().getCoordinate(), null, false);
         final ControllerServiceNode serviceNode3 = provider.createControllerService(ServiceA.class.getName(), "3",
-                systemBundle.getBundleDetails().getCoordinate(), false);
+                systemBundle.getBundleDetails().getCoordinate(), null, false);
         final ControllerServiceNode serviceNode4 = provider.createControllerService(ServiceB.class.getName(), "4",
-                systemBundle.getBundleDetails().getCoordinate(), false);
+                systemBundle.getBundleDetails().getCoordinate(), null, false);
 
         procGroup.addControllerService(serviceNode1);
         procGroup.addControllerService(serviceNode2);
@@ -251,9 +253,9 @@ public class TestStandardControllerServiceProvider {
         final StandardControllerServiceProvider provider =
                 new StandardControllerServiceProvider(controller, null, null, stateManagerProvider, variableRegistry, niFiProperties);
         final ControllerServiceNode serviceNode1 = provider.createControllerService(ServiceA.class.getName(), "1",
-                systemBundle.getBundleDetails().getCoordinate(), false);
+                systemBundle.getBundleDetails().getCoordinate(), null, false);
         final ControllerServiceNode serviceNode2 = provider.createControllerService(ServiceB.class.getName(), "2",
-                systemBundle.getBundleDetails().getCoordinate(), false);
+                systemBundle.getBundleDetails().getCoordinate(), null, false);
 
         setProperty(serviceNode1, ServiceA.OTHER_SERVICE.getName(), "2");
 
@@ -312,7 +314,7 @@ public class TestStandardControllerServiceProvider {
         // like that.
         nodeMap.clear();
         final ControllerServiceNode serviceNode3 = provider.createControllerService(ServiceA.class.getName(), "3",
-                systemBundle.getBundleDetails().getCoordinate(), false);
+                systemBundle.getBundleDetails().getCoordinate(), null, false);
         setProperty(serviceNode1, ServiceA.OTHER_SERVICE.getName(), "3");
         setProperty(serviceNode3, ServiceA.OTHER_SERVICE.getName(), "1");
         nodeMap.put("1", serviceNode1);
@@ -338,9 +340,9 @@ public class TestStandardControllerServiceProvider {
         nodeMap.clear();
         setProperty(serviceNode1, ServiceA.OTHER_SERVICE.getName(), "2");
         final ControllerServiceNode serviceNode4 = provider.createControllerService(ServiceB.class.getName(), "4",
-                systemBundle.getBundleDetails().getCoordinate(), false);
+                systemBundle.getBundleDetails().getCoordinate(), null, false);
         final ControllerServiceNode serviceNode5 = provider.createControllerService(ServiceB.class.getName(), "5",
-                systemBundle.getBundleDetails().getCoordinate(), false);
+                systemBundle.getBundleDetails().getCoordinate(), null, false);
         setProperty(serviceNode3, ServiceA.OTHER_SERVICE.getName(), "4");
         nodeMap.put("1", serviceNode1);
         nodeMap.put("2", serviceNode2);
@@ -397,10 +399,11 @@ public class TestStandardControllerServiceProvider {
     }
 
     private ProcessorNode createProcessor(final StandardProcessScheduler scheduler, final ControllerServiceProvider serviceProvider) {
+        final ReloadComponent reloadComponent = Mockito.mock(ReloadComponent.class);
         final LoggableComponent<Processor> dummyProcessor = new LoggableComponent<>(new DummyProcessor(), systemBundle.getBundleDetails().getCoordinate(), null);
         final ProcessorNode procNode = new StandardProcessorNode(dummyProcessor, UUID.randomUUID().toString(),
                 new StandardValidationContextFactory(serviceProvider, null), scheduler, serviceProvider, niFiProperties,
-                VariableRegistry.EMPTY_REGISTRY);
+                VariableRegistry.EMPTY_REGISTRY, reloadComponent);
 
         final ProcessGroup group = new StandardProcessGroup(UUID.randomUUID().toString(), serviceProvider, scheduler, null, null, null, variableRegistry);
         group.addProcessor(procNode);
@@ -419,7 +422,7 @@ public class TestStandardControllerServiceProvider {
         final StandardControllerServiceProvider provider =
                 new StandardControllerServiceProvider(controller, null, null, stateManagerProvider, variableRegistry, niFiProperties);
         final ControllerServiceNode serviceNode = provider.createControllerService(ServiceA.class.getName(), "1",
-                systemBundle.getBundleDetails().getCoordinate(), false);
+                systemBundle.getBundleDetails().getCoordinate(), null, false);
 
         final ProcessorNode procNode = createProcessor(scheduler, provider);
         serviceNode.addReference(procNode);
@@ -443,17 +446,17 @@ public class TestStandardControllerServiceProvider {
         Mockito.when(controller.getGroup(Mockito.anyString())).thenReturn(procGroup);
 
         ControllerServiceNode A = provider.createControllerService(ServiceA.class.getName(), "A",
-                systemBundle.getBundleDetails().getCoordinate(), false);
+                systemBundle.getBundleDetails().getCoordinate(), null, false);
         ControllerServiceNode B = provider.createControllerService(ServiceA.class.getName(), "B",
-                systemBundle.getBundleDetails().getCoordinate(), false);
+                systemBundle.getBundleDetails().getCoordinate(), null, false);
         ControllerServiceNode C = provider.createControllerService(ServiceA.class.getName(), "C",
-                systemBundle.getBundleDetails().getCoordinate(), false);
+                systemBundle.getBundleDetails().getCoordinate(), null, false);
         ControllerServiceNode D = provider.createControllerService(ServiceB.class.getName(), "D",
-                systemBundle.getBundleDetails().getCoordinate(), false);
+                systemBundle.getBundleDetails().getCoordinate(), null, false);
         ControllerServiceNode E = provider.createControllerService(ServiceA.class.getName(), "E",
-                systemBundle.getBundleDetails().getCoordinate(), false);
+                systemBundle.getBundleDetails().getCoordinate(), null, false);
         ControllerServiceNode F = provider.createControllerService(ServiceB.class.getName(), "F",
-                systemBundle.getBundleDetails().getCoordinate(), false);
+                systemBundle.getBundleDetails().getCoordinate(), null, false);
 
         procGroup.addControllerService(A);
         procGroup.addControllerService(B);
@@ -494,15 +497,15 @@ public class TestStandardControllerServiceProvider {
         Mockito.when(controller.getGroup(Mockito.anyString())).thenReturn(procGroup);
 
         ControllerServiceNode A = provider.createControllerService(ServiceC.class.getName(), "A",
-                systemBundle.getBundleDetails().getCoordinate(), false);
+                systemBundle.getBundleDetails().getCoordinate(), null, false);
         ControllerServiceNode B = provider.createControllerService(ServiceA.class.getName(), "B",
-                systemBundle.getBundleDetails().getCoordinate(), false);
+                systemBundle.getBundleDetails().getCoordinate(), null, false);
         ControllerServiceNode C = provider.createControllerService(ServiceB.class.getName(), "C",
-                systemBundle.getBundleDetails().getCoordinate(), false);
+                systemBundle.getBundleDetails().getCoordinate(), null, false);
         ControllerServiceNode D = provider.createControllerService(ServiceA.class.getName(), "D",
-                systemBundle.getBundleDetails().getCoordinate(), false);
+                systemBundle.getBundleDetails().getCoordinate(), null, false);
         ControllerServiceNode F = provider.createControllerService(ServiceA.class.getName(), "F",
-                systemBundle.getBundleDetails().getCoordinate(), false);
+                systemBundle.getBundleDetails().getCoordinate(), null, false);
 
         procGroup.addControllerService(A);
         procGroup.addControllerService(B);
@@ -536,19 +539,19 @@ public class TestStandardControllerServiceProvider {
         Mockito.when(controller.getGroup(Mockito.anyString())).thenReturn(procGroup);
 
         ControllerServiceNode serviceNode1 = provider.createControllerService(ServiceA.class.getName(), "1",
-                systemBundle.getBundleDetails().getCoordinate(), false);
+                systemBundle.getBundleDetails().getCoordinate(), null, false);
         ControllerServiceNode serviceNode2 = provider.createControllerService(ServiceA.class.getName(), "2",
-                systemBundle.getBundleDetails().getCoordinate(), false);
+                systemBundle.getBundleDetails().getCoordinate(), null, false);
         ControllerServiceNode serviceNode3 = provider.createControllerService(ServiceA.class.getName(), "3",
-                systemBundle.getBundleDetails().getCoordinate(), false);
+                systemBundle.getBundleDetails().getCoordinate(), null, false);
         ControllerServiceNode serviceNode4 = provider.createControllerService(ServiceB.class.getName(), "4",
-                systemBundle.getBundleDetails().getCoordinate(), false);
+                systemBundle.getBundleDetails().getCoordinate(), null, false);
         ControllerServiceNode serviceNode5 = provider.createControllerService(ServiceA.class.getName(), "5",
-                systemBundle.getBundleDetails().getCoordinate(), false);
+                systemBundle.getBundleDetails().getCoordinate(), null, false);
         ControllerServiceNode serviceNode6 = provider.createControllerService(ServiceB.class.getName(), "6",
-                systemBundle.getBundleDetails().getCoordinate(), false);
+                systemBundle.getBundleDetails().getCoordinate(), null, false);
         ControllerServiceNode serviceNode7 = provider.createControllerService(ServiceC.class.getName(), "7",
-                systemBundle.getBundleDetails().getCoordinate(), false);
+                systemBundle.getBundleDetails().getCoordinate(), null, false);
 
         procGroup.addControllerService(serviceNode1);
         procGroup.addControllerService(serviceNode2);

http://git-wip-us.apache.org/repos/asf/nifi/blob/556f309d/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/pom.xml
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/pom.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/pom.xml
new file mode 100644
index 0000000..ba20a37
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/pom.xml
@@ -0,0 +1,42 @@
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <!--
+      Licensed to the Apache Software Foundation (ASF) under one or more
+      contributor license agreements.  See the NOTICE file distributed with
+      this work for additional information regarding copyright ownership.
+      The ASF licenses this file to You under the Apache License, Version 2.0
+      (the "License"); you may not use this file except in compliance with
+      the License.  You may obtain a copy of the License at
+          http://www.apache.org/licenses/LICENSE-2.0
+      Unless required by applicable law or agreed to in writing, software
+      distributed under the License is distributed on an "AS IS" BASIS,
+      WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+      See the License for the specific language governing permissions and
+      limitations under the License.
+    -->
+    <modelVersion>4.0.0</modelVersion>
+    <parent>
+        <groupId>org.apache.nifi</groupId>
+        <artifactId>nifi-framework</artifactId>
+        <version>1.2.0-SNAPSHOT</version>
+    </parent>
+    <artifactId>nifi-framework-nar-utils</artifactId>
+    <packaging>jar</packaging>
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-nar-utils</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-properties</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-api</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-framework-api</artifactId>
+        </dependency>
+    </dependencies>
+</project>

http://git-wip-us.apache.org/repos/asf/nifi/blob/556f309d/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/init/ConfigurableComponentInitializer.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/init/ConfigurableComponentInitializer.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/init/ConfigurableComponentInitializer.java
new file mode 100644
index 0000000..9a1149c
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/init/ConfigurableComponentInitializer.java
@@ -0,0 +1,45 @@
+/*
+ * 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.init;
+
+import org.apache.nifi.components.ConfigurableComponent;
+import org.apache.nifi.reporting.InitializationException;
+
+/**
+ * An interface for initializing and tearing down a ConfigurableComponent. It is up to the
+ * implementer to call "init" so that you can call
+ * ConfigurableComponent.getPropertyDescriptors()
+ *
+ */
+public interface ConfigurableComponentInitializer {
+
+    /**
+     * Initializes a configurable component to the point that you can call
+     * getPropertyDescriptors() on it
+     *
+     * @param component the component to initialize
+     * @throws InitializationException if the component could not be initialized
+     */
+    void initialize(ConfigurableComponent component) throws InitializationException;
+
+    /**
+     * Calls the lifecycle methods that should be called when a flow is shutdown.
+     *
+     * @param component the component to initialize
+     */
+    void teardown(ConfigurableComponent component);
+}


[7/7] nifi git commit: NIFI-3520 Refactoring instance class loading - Fixing FlowController to use appropriate class loader when instantiating processor - Updating ExtensionManager to leverage new flag in MANIFEST from NAR plugin - Adding ReloadComponent

Posted by mc...@apache.org.
NIFI-3520 Refactoring instance class loading
- Fixing FlowController to use appropriate class loader when instantiating processor
- Updating ExtensionManager to leverage new flag in MANIFEST from NAR plugin
- Adding ReloadComponent interface and refactoring instance class loading to use it
- Fixing FetchHDFS issue with TDE by using ugi.doAs
- Refactoring nifi-nar-utils so that ExtensionManager only lives in nifi-framework
- Caching temp components found during service loader in ExtensionManager
- Updating authorizables, docs, and fingerprinting to use the cached components
- Introducing a flag on @RequiresInstanceClassLoading to indicate if ancestor resources should be cloned
- Updating developer guide regarding cloneAncestorResources flag
- This closes #1635


Project: http://git-wip-us.apache.org/repos/asf/nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/nifi/commit/556f309d
Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/556f309d
Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/556f309d

Branch: refs/heads/master
Commit: 556f309df086fefdcc6ca717294eaa91d3a4e113
Parents: 8f37ad4
Author: Bryan Bende <bb...@apache.org>
Authored: Fri Mar 24 17:14:24 2017 -0400
Committer: Matt Gilman <ma...@gmail.com>
Committed: Thu Apr 6 13:51:08 2017 -0400

----------------------------------------------------------------------
 .../behavior/RequiresInstanceClassLoading.java  |  18 +-
 .../src/main/asciidoc/developer-guide.adoc      |   5 +
 .../apache/nifi/bundle/BundleDetailsTest.java   |   2 +
 .../nifi-framework/nifi-documentation/pom.xml   |  12 +-
 .../apache/nifi/documentation/DocGenerator.java |  14 +-
 .../nifi/documentation/DocGeneratorTest.java    |   3 +-
 .../apache/nifi/cluster/integration/Node.java   |   3 +-
 .../nifi-framework-core-api/pom.xml             |   2 +-
 .../controller/AbstractConfiguredComponent.java |  41 +-
 .../nifi/controller/ConfiguredComponent.java    |   4 +
 .../apache/nifi/controller/ProcessorNode.java   |   4 +-
 .../apache/nifi/controller/ReloadComponent.java |  69 +++
 .../reporting/ReportingTaskProvider.java        |  10 -
 .../service/ControllerServiceProvider.java      |   4 +-
 .../nifi-framework/nifi-framework-core/pom.xml  |   8 +-
 .../apache/nifi/controller/FlowController.java  |  76 +--
 .../nifi/controller/StandardProcessorNode.java  |  19 +-
 .../reporting/AbstractReportingTaskNode.java    |  21 +-
 .../reporting/StandardReportingTaskNode.java    |  10 +-
 .../service/ControllerServiceLoader.java        |   6 +-
 .../service/StandardControllerServiceNode.java  |  22 +-
 .../StandardControllerServiceProvider.java      |  11 +-
 .../nifi/fingerprint/FingerprintFactory.java    |  82 +--
 .../nifi/groups/StandardProcessGroup.java       |   4 +-
 .../StandardFlowSynchronizerSpec.groovy         |   3 +-
 .../nifi/controller/TestFlowController.java     | 129 ++++-
 .../controller/TestStandardProcessorNode.java   | 180 ++++---
 .../scheduling/TestProcessorLifecycle.java      |   7 +-
 .../TestStandardProcessScheduler.java           |  27 +-
 .../StandardControllerServiceProviderTest.java  |   5 +-
 .../TestStandardControllerServiceProvider.java  |  69 +--
 .../nifi-framework-nar-utils/pom.xml            |  42 ++
 .../init/ConfigurableComponentInitializer.java  |  45 ++
 ...ConfigurableComponentInitializerFactory.java |  44 ++
 .../nifi/init/ControllerServiceInitializer.java |  59 ++
 .../apache/nifi/init/ProcessorInitializer.java  |  58 ++
 .../org/apache/nifi/init/ReflectionUtils.java   | 133 +++++
 .../nifi/init/ReportingTaskingInitializer.java  |  57 ++
 .../apache/nifi/mock/MockComponentLogger.java   | 258 +++++++++
 .../nifi/mock/MockConfigurationContext.java     |  48 ++
 ...kControllerServiceInitializationContext.java |  68 +++
 .../nifi/mock/MockControllerServiceLookup.java  |  63 +++
 .../apache/nifi/mock/MockNodeTypeProvider.java  |  40 ++
 .../apache/nifi/mock/MockProcessContext.java    | 116 ++++
 .../MockProcessorInitializationContext.java     |  68 +++
 .../MockReportingInitializationContext.java     |  83 +++
 .../org/apache/nifi/nar/ExtensionManager.java   | 537 +++++++++++++++++++
 .../apache/nifi/nar/InstanceClassLoader.java    |  89 +++
 .../java/org/apache/nifi/nar/NarCloseable.java  | 112 ++++
 .../nifi/nar/NarThreadContextClassLoader.java   | 235 ++++++++
 .../nar/NarThreadContextClassLoaderTest.java    |  99 ++++
 .../org/apache/nifi/nar/NarUnpackerTest.java    | 183 +++++++
 .../org.apache.nifi.processor.Processor         |  16 +
 .../resources/NarUnpacker/conf/nifi.properties  | 125 +++++
 .../resources/NarUnpacker/lib/dummy-one.nar     | Bin 0 -> 1749 bytes
 .../NarUnpacker/lib/nifi-framework-nar.nar      | Bin 0 -> 406 bytes
 .../resources/NarUnpacker/lib2/dummy-two.nar    | Bin 0 -> 1751 bytes
 .../src/test/resources/nifi.properties          | 198 +++++++
 .../nifi-framework/nifi-nar-utils/pom.xml       |   1 +
 .../init/ConfigurableComponentInitializer.java  |  45 --
 ...ConfigurableComponentInitializerFactory.java |  44 --
 .../nifi/init/ControllerServiceInitializer.java |  59 --
 .../apache/nifi/init/ProcessorInitializer.java  |  58 --
 .../org/apache/nifi/init/ReflectionUtils.java   | 133 -----
 .../nifi/init/ReportingTaskingInitializer.java  |  57 --
 .../apache/nifi/mock/MockComponentLogger.java   | 258 ---------
 .../nifi/mock/MockConfigurationContext.java     |  48 --
 ...kControllerServiceInitializationContext.java |  68 ---
 .../nifi/mock/MockControllerServiceLookup.java  |  63 ---
 .../apache/nifi/mock/MockNodeTypeProvider.java  |  40 --
 .../apache/nifi/mock/MockProcessContext.java    | 116 ----
 .../MockProcessorInitializationContext.java     |  68 ---
 .../MockReportingInitializationContext.java     |  83 ---
 .../org/apache/nifi/nar/ExtensionManager.java   | 478 -----------------
 .../apache/nifi/nar/InstanceClassLoader.java    | 160 ------
 .../org/apache/nifi/nar/NarClassLoaders.java    |   1 +
 .../java/org/apache/nifi/nar/NarCloseable.java  | 112 ----
 .../org/apache/nifi/nar/NarManifestEntry.java   |   2 +-
 .../nifi/nar/NarThreadContextClassLoader.java   | 235 --------
 .../java/org/apache/nifi/nar/SystemBundle.java  |  57 ++
 .../org/apache/nifi/nar/NarBundleUtilTest.java  |   6 +-
 .../nar/NarThreadContextClassLoaderTest.java    |  99 ----
 .../org/apache/nifi/nar/NarUnpackerTest.java    | 183 -------
 .../org.apache.nifi.processor.Processor         |  16 -
 .../resources/NarUnpacker/conf/nifi.properties  | 124 -----
 .../resources/NarUnpacker/lib/dummy-one.nar     | Bin 1749 -> 0 bytes
 .../NarUnpacker/lib/nifi-framework-nar.nar      | Bin 406 -> 0 bytes
 .../resources/NarUnpacker/lib2/dummy-two.nar    | Bin 1751 -> 0 bytes
 .../nifi-framework/nifi-runtime/pom.xml         |   5 -
 .../src/main/java/org/apache/nifi/NiFi.java     |  11 +-
 .../main/java/org/apache/nifi/NiFiServer.java   |   5 +
 .../nifi-framework/nifi-web/nifi-jetty/pom.xml  |   5 +
 .../org/apache/nifi/web/server/JettyServer.java |  16 +
 .../nifi/authorization/AuthorizableLookup.java  |  43 +-
 .../AuthorizeControllerServiceReference.java    |   6 +-
 .../authorization/ComponentAuthorizable.java    |  69 +++
 .../ConfigurableComponentAuthorizable.java      |  69 ---
 .../authorization/ProcessGroupAuthorizable.java |   4 +-
 .../nifi/authorization/SnippetAuthorizable.java |   2 +-
 .../StandardAuthorizableLookup.java             | 142 +++--
 .../TemplateContentsAuthorizable.java           |   4 +-
 .../nifi/web/StandardNiFiServiceFacade.java     |  10 +-
 .../StandardNiFiWebConfigurationContext.java    |   8 +-
 .../apache/nifi/web/api/ControllerResource.java |  10 +-
 .../nifi/web/api/ControllerServiceResource.java |   6 +-
 .../nifi/web/api/ProcessGroupResource.java      |  14 +-
 .../apache/nifi/web/api/ProcessorResource.java  |   6 +-
 .../nifi/web/api/ReportingTaskResource.java     |   6 +-
 .../nifi/web/controller/ControllerFacade.java   |  40 +-
 .../dao/impl/StandardControllerServiceDAO.java  |   5 +-
 .../nifi/web/dao/impl/StandardProcessorDAO.java |   3 +-
 .../web/dao/impl/StandardReportingTaskDAO.java  |   9 +-
 .../src/main/resources/nifi-web-api-context.xml |   1 +
 .../accesscontrol/AccessControlHelper.java      |   3 +-
 .../accesscontrol/ITAccessTokenEndpoint.java    |   3 +-
 .../nifi/web/StandardNiFiServiceFacadeTest.java |   4 +-
 .../nifi-framework/pom.xml                      |   1 +
 nifi-nar-bundles/nifi-framework-bundle/pom.xml  |   5 +
 .../nifi-hadoop-bundle/nifi-hadoop-nar/pom.xml  |   2 +-
 .../nifi-hdfs-processors/pom.xml                |  12 +-
 .../hadoop/AbstractHadoopProcessor.java         |  33 +-
 .../nifi/processors/hadoop/FetchHDFS.java       | 103 ++--
 .../nifi/controller/MonitorMemoryTest.java      |   3 +-
 123 files changed, 3763 insertions(+), 3165 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/nifi/blob/556f309d/nifi-api/src/main/java/org/apache/nifi/annotation/behavior/RequiresInstanceClassLoading.java
----------------------------------------------------------------------
diff --git a/nifi-api/src/main/java/org/apache/nifi/annotation/behavior/RequiresInstanceClassLoading.java b/nifi-api/src/main/java/org/apache/nifi/annotation/behavior/RequiresInstanceClassLoading.java
index f7566a6..0796199 100644
--- a/nifi-api/src/main/java/org/apache/nifi/annotation/behavior/RequiresInstanceClassLoading.java
+++ b/nifi-api/src/main/java/org/apache/nifi/annotation/behavior/RequiresInstanceClassLoading.java
@@ -28,10 +28,19 @@ import java.lang.annotation.Target;
  *  for each instance of the component, copying all resources from the component's NARClassLoader to a
  *  new ClassLoader which will only be used by a given instance of the component.
  *
- *  This annotation is typically used when a component has one or more PropertyDescriptors which set
- *  dynamicallyModifiesClasspath(boolean) to true.
+ *  If cloneAncestorResources is set to true, the instance ClassLoader will include ancestor resources up to the
+ *  first ClassLoader containing a controller service API referenced by the component, or up to the Jetty NAR.
  *
- *  When this annotation is used it is important to note that each added instance of the component will increase
+ *  Example #1 - PutHDFS has this flag set to true and does not reference any controller services, so it will include
+ *  resources from nifi-hadoop-nar, nifi-hadoop-libraries-nar, and nifi-standard-services-api-nar, stopping at nifi-jetty-nar.
+ *
+ *  Example #2 - If PutHDFS referenced an SSLContext and has this flag set to true, then it would include
+ *  resources from nifi-hadoop-nar, nifi-hadoop-libraries-nar, and stop before nifi-standard-services-api-nar.
+ *
+ *  Example #3 - HBaseClientService_1_1_2 does not have this flag set so it defaults to false, and therefore includes
+ *  only resources from the nifi-hbase-client-service-1_1_2-nar.
+ *
+ *  NOTE: When this annotation is used it is important to note that each added instance of the component will increase
  *  the overall memory footprint more than that of a component without this annotation.
  */
 @Documented
@@ -39,4 +48,7 @@ import java.lang.annotation.Target;
 @Retention(RetentionPolicy.RUNTIME)
 @Inherited
 public @interface RequiresInstanceClassLoading {
+
+    boolean cloneAncestorResources() default false;
+
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/556f309d/nifi-docs/src/main/asciidoc/developer-guide.adoc
----------------------------------------------------------------------
diff --git a/nifi-docs/src/main/asciidoc/developer-guide.adoc b/nifi-docs/src/main/asciidoc/developer-guide.adoc
index 429bd2c..7b2ba35 100644
--- a/nifi-docs/src/main/asciidoc/developer-guide.adoc
+++ b/nifi-docs/src/main/asciidoc/developer-guide.adoc
@@ -2357,6 +2357,11 @@ component's NAR ClassLoader. When `@RequiresInstanceClassLoading` is not present
 instance ClassLoader simply has it's parent ClassLoader set to the NAR ClassLoader, rather than
 copying resources.
 
+The `@RequiresInstanceClassLoading` annotation also provides an optional flag `cloneAncestorResources'. If
+set to true, the instance ClassLoader will include ancestor resources up to the first ClassLoader containing a
+controller service API referenced by the component, or up to the Jetty NAR. If set to false, or not specified,
+only the resources from the component's NAR will be included.
+
 Because @RequiresInstanceClassLoading copies resources from the NAR ClassLoader for each instance of the
 component, use this capability judiciously. If ten instances of one component are created, all classes
 from the component's NAR ClassLoader are loaded into memory ten times. This could eventually increase the

http://git-wip-us.apache.org/repos/asf/nifi/blob/556f309d/nifi-framework-api/src/test/java/org/apache/nifi/bundle/BundleDetailsTest.java
----------------------------------------------------------------------
diff --git a/nifi-framework-api/src/test/java/org/apache/nifi/bundle/BundleDetailsTest.java b/nifi-framework-api/src/test/java/org/apache/nifi/bundle/BundleDetailsTest.java
index e2475b6..62765dd 100644
--- a/nifi-framework-api/src/test/java/org/apache/nifi/bundle/BundleDetailsTest.java
+++ b/nifi-framework-api/src/test/java/org/apache/nifi/bundle/BundleDetailsTest.java
@@ -38,6 +38,8 @@ public class BundleDetailsTest {
         final String buildJdk = "JDK8";
         final String builtBy = "bbende";
 
+        final boolean cloneDuringInstanceClassLoading = true;
+
         final BundleDetails bundleDetails = new BundleDetails.Builder()
                 .workingDir(workingDirectory)
                 .coordinate(coordinate)

http://git-wip-us.apache.org/repos/asf/nifi/blob/556f309d/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/pom.xml
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/pom.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/pom.xml
index 01467ab..0e02174 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/pom.xml
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/pom.xml
@@ -20,10 +20,6 @@
     <dependencies>
         <dependency>
             <groupId>org.apache.nifi</groupId>
-            <artifactId>nifi-nar-utils</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-api</artifactId>
         </dependency>
         <dependency>
@@ -36,6 +32,14 @@
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-nar-utils</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-framework-nar-utils</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-utils</artifactId>
             <scope>test</scope>
         </dependency>

http://git-wip-us.apache.org/repos/asf/nifi/blob/556f309d/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/DocGenerator.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/DocGenerator.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/DocGenerator.java
index 9ae1705..dd99927 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/DocGenerator.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/DocGenerator.java
@@ -22,8 +22,6 @@ import org.apache.nifi.components.ConfigurableComponent;
 import org.apache.nifi.controller.ControllerService;
 import org.apache.nifi.documentation.html.HtmlDocumentationWriter;
 import org.apache.nifi.documentation.html.HtmlProcessorDocumentationWriter;
-import org.apache.nifi.init.ConfigurableComponentInitializer;
-import org.apache.nifi.init.ConfigurableComponentInitializerFactory;
 import org.apache.nifi.nar.ExtensionManager;
 import org.apache.nifi.nar.ExtensionMapping;
 import org.apache.nifi.processor.Processor;
@@ -93,7 +91,7 @@ public class DocGenerator {
                 final Class<? extends ConfigurableComponent> componentClass = extensionClass.asSubclass(ConfigurableComponent.class);
                 try {
                     logger.debug("Documenting: " + componentClass);
-                    document(componentDirectory, componentClass);
+                    document(componentDirectory, componentClass, coordinate);
                 } catch (Exception e) {
                     logger.warn("Unable to document: " + componentClass, e);
                 }
@@ -113,12 +111,12 @@ public class DocGenerator {
      * @throws IOException ioe
      * @throws InitializationException ie
      */
-    private static void document(final File componentDocsDir, final Class<? extends ConfigurableComponent> componentClass)
+    private static void document(final File componentDocsDir, final Class<? extends ConfigurableComponent> componentClass, final BundleCoordinate bundleCoordinate)
             throws InstantiationException, IllegalAccessException, IOException, InitializationException {
 
-        final ConfigurableComponent component = componentClass.newInstance();
-        final ConfigurableComponentInitializer initializer = ConfigurableComponentInitializerFactory.createComponentInitializer(componentClass);
-        initializer.initialize(component);
+        // use temp components from ExtensionManager which should always be populated before doc generation
+        final String classType = componentClass.getCanonicalName();
+        final ConfigurableComponent component = ExtensionManager.getTempComponent(classType, bundleCoordinate);
 
         final DocumentationWriter writer = getDocumentWriter(componentClass);
 
@@ -130,8 +128,6 @@ public class DocGenerator {
         try (final OutputStream output = new BufferedOutputStream(new FileOutputStream(baseDocumentationFile))) {
             writer.write(component, output, hasAdditionalInfo(componentDocsDir));
         }
-
-        initializer.teardown(component);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/nifi/blob/556f309d/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/java/org/apache/nifi/documentation/DocGeneratorTest.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/java/org/apache/nifi/documentation/DocGeneratorTest.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/java/org/apache/nifi/documentation/DocGeneratorTest.java
index b00ab8a..a165de1 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/java/org/apache/nifi/documentation/DocGeneratorTest.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/java/org/apache/nifi/documentation/DocGeneratorTest.java
@@ -23,6 +23,7 @@ import org.apache.nifi.nar.ExtensionManager;
 import org.apache.nifi.nar.ExtensionMapping;
 import org.apache.nifi.nar.NarClassLoaders;
 import org.apache.nifi.nar.NarUnpacker;
+import org.apache.nifi.nar.SystemBundle;
 import org.apache.nifi.util.NiFiProperties;
 import org.junit.Assert;
 import org.junit.Test;
@@ -47,7 +48,7 @@ public class DocGeneratorTest {
                 NiFiProperties.COMPONENT_DOCS_DIRECTORY,
                 temporaryFolder.getRoot().getAbsolutePath());
 
-        final Bundle systemBundle = ExtensionManager.createSystemBundle(properties);
+        final Bundle systemBundle = SystemBundle.create(properties);
         final ExtensionMapping mapping = NarUnpacker.unpackNars(properties, systemBundle);
 
         NarClassLoaders.getInstance().init(properties.getFrameworkWorkingDirectory(), properties.getExtensionsWorkingDirectory());

http://git-wip-us.apache.org/repos/asf/nifi/blob/556f309d/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/integration/Node.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/integration/Node.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/integration/Node.java
index 20dbfe1..cace715 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/integration/Node.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/integration/Node.java
@@ -64,6 +64,7 @@ import org.apache.nifi.events.EventReporter;
 import org.apache.nifi.io.socket.ServerSocketConfiguration;
 import org.apache.nifi.io.socket.SocketConfiguration;
 import org.apache.nifi.nar.ExtensionManager;
+import org.apache.nifi.nar.SystemBundle;
 import org.apache.nifi.registry.VariableRegistry;
 import org.apache.nifi.reporting.BulletinRepository;
 import org.apache.nifi.reporting.Severity;
@@ -120,7 +121,7 @@ public class Node {
             }
         };
 
-        final Bundle systemBundle = ExtensionManager.createSystemBundle(properties);
+        final Bundle systemBundle = SystemBundle.create(properties);
         ExtensionManager.discoverExtensions(systemBundle, Collections.emptySet());
 
         revisionManager = Mockito.mock(RevisionManager.class);

http://git-wip-us.apache.org/repos/asf/nifi/blob/556f309d/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/pom.xml
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/pom.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/pom.xml
index 78d895e..c3bb182 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/pom.xml
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/pom.xml
@@ -24,7 +24,7 @@
     <dependencies>
         <dependency>
             <groupId>org.apache.nifi</groupId>
-            <artifactId>nifi-nar-utils</artifactId>
+            <artifactId>nifi-framework-nar-utils</artifactId>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>

http://git-wip-us.apache.org/repos/asf/nifi/blob/556f309d/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/AbstractConfiguredComponent.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/AbstractConfiguredComponent.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/AbstractConfiguredComponent.java
index 345ce64..1156987 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/AbstractConfiguredComponent.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/AbstractConfiguredComponent.java
@@ -27,12 +27,10 @@ import org.apache.nifi.components.ValidationResult;
 import org.apache.nifi.controller.service.ControllerServiceNode;
 import org.apache.nifi.controller.service.ControllerServiceProvider;
 import org.apache.nifi.nar.ExtensionManager;
-import org.apache.nifi.nar.InstanceClassLoader;
 import org.apache.nifi.nar.NarCloseable;
 import org.apache.nifi.registry.VariableRegistry;
 import org.apache.nifi.util.file.classloader.ClassLoaderUtils;
 
-import java.net.MalformedURLException;
 import java.net.URL;
 import java.util.ArrayList;
 import java.util.Collection;
@@ -60,6 +58,7 @@ public abstract class AbstractConfiguredComponent implements ConfigurableCompone
     private final String componentType;
     private final String componentCanonicalClass;
     private final VariableRegistry variableRegistry;
+    private final ReloadComponent reloadComponent;
 
     private final AtomicBoolean isExtensionMissing;
 
@@ -69,7 +68,7 @@ public abstract class AbstractConfiguredComponent implements ConfigurableCompone
     public AbstractConfiguredComponent(final String id,
                                        final ValidationContextFactory validationContextFactory, final ControllerServiceProvider serviceProvider,
                                        final String componentType, final String componentCanonicalClass, final VariableRegistry variableRegistry,
-                                       final boolean isExtensionMissing) {
+                                       final ReloadComponent reloadComponent, final boolean isExtensionMissing) {
         this.id = id;
         this.validationContextFactory = validationContextFactory;
         this.serviceProvider = serviceProvider;
@@ -78,6 +77,7 @@ public abstract class AbstractConfiguredComponent implements ConfigurableCompone
         this.componentCanonicalClass = componentCanonicalClass;
         this.variableRegistry = variableRegistry;
         this.isExtensionMissing = new AtomicBoolean(isExtensionMissing);
+        this.reloadComponent = reloadComponent;
     }
 
     @Override
@@ -234,37 +234,28 @@ public abstract class AbstractConfiguredComponent implements ConfigurableCompone
     }
 
     /**
-     * Adds all of the modules identified by the given module paths to the InstanceClassLoader for this component.
+     * Triggers the reloading of the underlying component using a new InstanceClassLoader that includes the additional URL resources.
      *
      * @param modulePaths a list of module paths where each entry can be a comma-separated list of multiple module paths
      */
     private void processClasspathModifiers(final Set<String> modulePaths) {
         try {
+            // compute the URLs from all the modules paths
             final URL[] urls = ClassLoaderUtils.getURLsForClasspath(modulePaths, null, true);
 
-            if (getLogger().isDebugEnabled()) {
-                getLogger().debug("Adding {} resources to the classpath for {}", new Object[] {urls.length, name});
-                for (URL url : urls) {
-                    getLogger().debug(url.getFile());
+            // convert to a set of URLs
+            final Set<URL> additionalUrls = new LinkedHashSet<>();
+            if (urls != null) {
+                for (final URL url : urls) {
+                    additionalUrls.add(url);
                 }
             }
 
-            final ClassLoader classLoader = Thread.currentThread().getContextClassLoader();
+            // reload the underlying component with a new InstanceClassLoader that includes the new URLs
+            reload(additionalUrls);
 
-            if (!(classLoader instanceof InstanceClassLoader)) {
-                // Really shouldn't happen, but if we somehow got here and don't have an InstanceClassLoader then log a warning and move on
-                final String classLoaderName = classLoader == null ? "null" : classLoader.getClass().getName();
-                if (getLogger().isWarnEnabled()) {
-                    getLogger().warn(String.format("Unable to modify the classpath for %s, expected InstanceClassLoader, but found %s", name, classLoaderName));
-                }
-                return;
-            }
-
-            final InstanceClassLoader instanceClassLoader = (InstanceClassLoader) classLoader;
-            instanceClassLoader.setInstanceResources(urls);
-        } catch (MalformedURLException e) {
-            // Shouldn't get here since we are suppressing errors
-            getLogger().warn("Error processing classpath resources", e);
+        } catch (Exception e) {
+            getLogger().warn("Error processing classpath resources for " + id + ": " + e.getMessage(), e);
         }
     }
 
@@ -506,6 +497,10 @@ public abstract class AbstractConfiguredComponent implements ConfigurableCompone
         return this.variableRegistry;
     }
 
+    protected ReloadComponent getReloadComponent() {
+        return this.reloadComponent;
+    }
+
     @Override
     public void verifyCanUpdateBundle(final BundleCoordinate incomingCoordinate) throws IllegalArgumentException {
         final BundleCoordinate existingCoordinate = getBundleCoordinate();

http://git-wip-us.apache.org/repos/asf/nifi/blob/556f309d/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/ConfiguredComponent.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/ConfiguredComponent.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/ConfiguredComponent.java
index 1c0b7c1..2ccb858 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/ConfiguredComponent.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/ConfiguredComponent.java
@@ -30,8 +30,10 @@ import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.ValidationResult;
 import org.apache.nifi.logging.ComponentLog;
 
+import java.net.URL;
 import java.util.Collection;
 import java.util.Map;
+import java.util.Set;
 
 public interface ConfiguredComponent extends ComponentAuthorizable {
 
@@ -53,6 +55,8 @@ public interface ConfiguredComponent extends ComponentAuthorizable {
 
     boolean isValid();
 
+    void reload(Set<URL> additionalUrls) throws Exception;
+
     BundleCoordinate getBundleCoordinate();
 
     ConfigurableComponent getComponent();

http://git-wip-us.apache.org/repos/asf/nifi/blob/556f309d/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/ProcessorNode.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/ProcessorNode.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/ProcessorNode.java
index 8c78958..aac5e52 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/ProcessorNode.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/ProcessorNode.java
@@ -47,8 +47,8 @@ public abstract class ProcessorNode extends AbstractConfiguredComponent implemen
     public ProcessorNode(final String id,
                          final ValidationContextFactory validationContextFactory, final ControllerServiceProvider serviceProvider,
                          final String componentType, final String componentCanonicalClass, final VariableRegistry variableRegistry,
-                         final boolean isExtensionMissing) {
-        super(id, validationContextFactory, serviceProvider, componentType, componentCanonicalClass, variableRegistry, isExtensionMissing);
+                         final ReloadComponent reloadComponent, final boolean isExtensionMissing) {
+        super(id, validationContextFactory, serviceProvider, componentType, componentCanonicalClass, variableRegistry, reloadComponent, isExtensionMissing);
         this.scheduledState = new AtomicReference<>(ScheduledState.STOPPED);
     }
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/556f309d/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/ReloadComponent.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/ReloadComponent.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/ReloadComponent.java
new file mode 100644
index 0000000..6886ef6
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/ReloadComponent.java
@@ -0,0 +1,69 @@
+/*
+ * 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.controller;
+
+import org.apache.nifi.bundle.BundleCoordinate;
+import org.apache.nifi.controller.exception.ControllerServiceInstantiationException;
+import org.apache.nifi.controller.exception.ProcessorInstantiationException;
+import org.apache.nifi.controller.reporting.ReportingTaskInstantiationException;
+import org.apache.nifi.controller.service.ControllerServiceNode;
+
+import java.net.URL;
+import java.util.Set;
+
+/**
+ * A service used to change the type of an underlying component.
+ */
+public interface ReloadComponent {
+
+    /**
+     * Changes the underlying Processor held by the node to an instance of the new type.
+     *
+     * @param existingNode the node being being updated
+     * @param newType the fully qualified class name of the new type
+     * @param bundleCoordinate the bundle coordinate of the new type
+     * @param additionalUrls additional URLs to be added to the instance class loader of the new component
+     * @throws ControllerServiceInstantiationException if unable to create an instance of the new type
+     */
+    void reload(ProcessorNode existingNode, String newType, BundleCoordinate bundleCoordinate, Set<URL> additionalUrls)
+            throws ProcessorInstantiationException;
+
+    /**
+     * Changes the underlying ControllerService held by the node to an instance of the new type.
+     *
+     * @param existingNode the node being being updated
+     * @param newType the fully qualified class name of the new type
+     * @param bundleCoordinate the bundle coordinate of the new type
+     * @param additionalUrls additional URLs to be added to the instance class loader of the new component
+     * @throws ControllerServiceInstantiationException if unable to create an instance of the new type
+     */
+    void reload(ControllerServiceNode existingNode, String newType, BundleCoordinate bundleCoordinate, Set<URL> additionalUrls)
+            throws ControllerServiceInstantiationException;
+
+    /**
+     * Changes the underlying ReportingTask held by the node to an instance of the new type.
+     *
+     * @param existingNode the ReportingTaskNode being updated
+     * @param newType the fully qualified class name of the new type
+     * @param bundleCoordinate the bundle coordinate of the new type
+     * @param additionalUrls additional URLs to be added to the instance class loader of the new component
+     * @throws ReportingTaskInstantiationException if unable to create an instance of the new type
+     */
+    void reload(ReportingTaskNode existingNode, String newType, BundleCoordinate bundleCoordinate, Set<URL> additionalUrls)
+            throws ReportingTaskInstantiationException;
+
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/556f309d/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/reporting/ReportingTaskProvider.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/reporting/ReportingTaskProvider.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/reporting/ReportingTaskProvider.java
index b48f198..c826775 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/reporting/ReportingTaskProvider.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/reporting/ReportingTaskProvider.java
@@ -112,14 +112,4 @@ public interface ReportingTaskProvider {
      */
     void disableReportingTask(ReportingTaskNode reportingTask);
 
-    /**
-     * Changes the underlying ReportingTask held by the node to an instance of the new type.
-     *
-     * @param reportingTask the ReportingTaskNode being updated
-     * @param newType the fully qualified class name of the new type
-     * @param bundleCoordinate the bundle coordinate of the new type
-     * @throws ReportingTaskInstantiationException if unable to create an instance of the new type
-     */
-    void changeReportingTaskType(final ReportingTaskNode reportingTask, final String newType, final BundleCoordinate bundleCoordinate) throws ReportingTaskInstantiationException;
-
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/556f309d/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/service/ControllerServiceProvider.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/service/ControllerServiceProvider.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/service/ControllerServiceProvider.java
index d1111e6..4169281 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/service/ControllerServiceProvider.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/service/ControllerServiceProvider.java
@@ -16,6 +16,7 @@
  */
 package org.apache.nifi.controller.service;
 
+import java.net.URL;
 import java.util.Collection;
 import java.util.Set;
 
@@ -38,10 +39,11 @@ public interface ControllerServiceProvider extends ControllerServiceLookup {
      * @param type of service
      * @param id of service
      * @param bundleCoordinate the coordinate of the bundle for the service
+     * @param additionalUrls optional additional URL resources to add to the class loader of the component
      * @param firstTimeAdded for service
      * @return the service node
      */
-    ControllerServiceNode createControllerService(String type, String id, BundleCoordinate bundleCoordinate, boolean firstTimeAdded);
+    ControllerServiceNode createControllerService(String type, String id, BundleCoordinate bundleCoordinate, Set<URL> additionalUrls, boolean firstTimeAdded);
 
     /**
      * @param id of the service

http://git-wip-us.apache.org/repos/asf/nifi/blob/556f309d/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/pom.xml
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/pom.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/pom.xml
index 3e92219..54d777f 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/pom.xml
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/pom.xml
@@ -41,19 +41,19 @@
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
-            <artifactId>nifi-expression-language</artifactId>
+            <artifactId>nifi-nar-utils</artifactId>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
-            <artifactId>nifi-utils</artifactId>
+            <artifactId>nifi-expression-language</artifactId>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
-            <artifactId>nifi-schema-utils</artifactId>
+            <artifactId>nifi-utils</artifactId>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
-            <artifactId>nifi-nar-utils</artifactId>
+            <artifactId>nifi-schema-utils</artifactId>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>

http://git-wip-us.apache.org/repos/asf/nifi/blob/556f309d/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java
index f312096..151640e 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java
@@ -221,6 +221,7 @@ import java.io.ByteArrayInputStream;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
+import java.net.URL;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
@@ -247,7 +248,7 @@ import java.util.stream.Collectors;
 import static java.util.Objects.requireNonNull;
 
 public class FlowController implements EventAccess, ControllerServiceProvider, ReportingTaskProvider,
-    QueueProvider, Authorizable, ProvenanceAuthorizableFactory, NodeTypeProvider, IdentifierLookup {
+    QueueProvider, Authorizable, ProvenanceAuthorizableFactory, NodeTypeProvider, IdentifierLookup, ReloadComponent {
 
     // default repository implementations
     public static final String DEFAULT_FLOWFILE_REPO_IMPLEMENTATION = "org.apache.nifi.controller.repository.WriteAheadFlowFileRepository";
@@ -1049,7 +1050,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
      * instantiated for any reason
      */
     public ProcessorNode createProcessor(final String type, String id, final BundleCoordinate coordinate, final boolean firstTimeAdded) throws ProcessorInstantiationException {
-        return createProcessor(type, id, coordinate, firstTimeAdded, true);
+        return createProcessor(type, id, coordinate, Collections.emptySet(), firstTimeAdded, true);
     }
 
     /**
@@ -1069,14 +1070,14 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
      * @throws ProcessorInstantiationException if the processor cannot be
      * instantiated for any reason
      */
-    public ProcessorNode createProcessor(final String type, String id, final BundleCoordinate coordinate, final boolean firstTimeAdded, final boolean registerLogObserver)
-            throws ProcessorInstantiationException {
+    public ProcessorNode createProcessor(final String type, String id, final BundleCoordinate coordinate, final Set<URL> additionalUrls,
+                                         final boolean firstTimeAdded, final boolean registerLogObserver) throws ProcessorInstantiationException {
         id = id.intern();
 
         boolean creationSuccessful;
         LoggableComponent<Processor> processor;
         try {
-            processor = instantiateProcessor(type, id, coordinate);
+            processor = instantiateProcessor(type, id, coordinate, additionalUrls);
             creationSuccessful = true;
         } catch (final ProcessorInstantiationException pie) {
             LOG.error("Could not create Processor of type " + type + " for ID " + id + "; creating \"Ghost\" implementation", pie);
@@ -1090,12 +1091,12 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
         final ValidationContextFactory validationContextFactory = new StandardValidationContextFactory(controllerServiceProvider, variableRegistry);
         final ProcessorNode procNode;
         if (creationSuccessful) {
-            procNode = new StandardProcessorNode(processor, id, validationContextFactory, processScheduler, controllerServiceProvider, nifiProperties, variableRegistry);
+            procNode = new StandardProcessorNode(processor, id, validationContextFactory, processScheduler, controllerServiceProvider, nifiProperties, variableRegistry, this);
         } else {
             final String simpleClassName = type.contains(".") ? StringUtils.substringAfterLast(type, ".") : type;
             final String componentType = "(Missing) " + simpleClassName;
             procNode = new StandardProcessorNode(
-                    processor, id, validationContextFactory, processScheduler, controllerServiceProvider, componentType, type, nifiProperties, variableRegistry, true);
+                    processor, id, validationContextFactory, processScheduler, controllerServiceProvider, componentType, type, nifiProperties, variableRegistry, this, true);
         }
 
         final LogRepository logRepository = LogRepositoryFactory.getRepository(id);
@@ -1153,7 +1154,9 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
         return procNode;
     }
 
-    private LoggableComponent<Processor> instantiateProcessor(final String type, final String identifier, final BundleCoordinate bundleCoordinate) throws ProcessorInstantiationException {
+    private LoggableComponent<Processor> instantiateProcessor(final String type, final String identifier, final BundleCoordinate bundleCoordinate, final Set<URL> additionalUrls)
+            throws ProcessorInstantiationException {
+
         final Bundle processorBundle = ExtensionManager.getBundle(bundleCoordinate);
         if (processorBundle == null) {
             throw new ProcessorInstantiationException("Unable to find bundle for coordinate " + bundleCoordinate.getCoordinate());
@@ -1161,9 +1164,9 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
 
         final ClassLoader ctxClassLoader = Thread.currentThread().getContextClassLoader();
         try {
-            final ClassLoader detectedClassLoaderForType = ExtensionManager.createInstanceClassLoader(type, identifier, processorBundle);
-            final Class<?> rawClass = Class.forName(type, true, processorBundle.getClassLoader());
-            Thread.currentThread().setContextClassLoader(detectedClassLoaderForType);
+            final ClassLoader detectedClassLoaderForInstance = ExtensionManager.createInstanceClassLoader(type, identifier, processorBundle, additionalUrls);
+            final Class<?> rawClass = Class.forName(type, true, detectedClassLoaderForInstance);
+            Thread.currentThread().setContextClassLoader(detectedClassLoaderForInstance);
 
             final Class<? extends Processor> processorClass = rawClass.asSubclass(Processor.class);
             final Processor processor = processorClass.newInstance();
@@ -1184,7 +1187,9 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
         }
     }
 
-    public void changeProcessorType(final ProcessorNode existingNode, final String newType, final BundleCoordinate bundleCoordinate) throws ProcessorInstantiationException {
+    @Override
+    public void reload(final ProcessorNode existingNode, final String newType, final BundleCoordinate bundleCoordinate, final Set<URL> additionalUrls)
+            throws ProcessorInstantiationException {
         if (existingNode == null) {
             throw new IllegalStateException("Existing ProcessorNode cannot be null");
         }
@@ -1197,13 +1202,15 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
 
         // create a new node with firstTimeAdded as true so lifecycle methods get fired
         // attempt the creation to make sure it works before firing the OnRemoved methods below
-        final ProcessorNode newNode = createProcessor(newType, id, bundleCoordinate, true, false);
+        final ProcessorNode newNode = createProcessor(newType, id, bundleCoordinate, additionalUrls, true, false);
 
         // call OnRemoved for the existing processor using the previous instance class loader
         try (final NarCloseable x = NarCloseable.withComponentNarLoader(existingInstanceClassLoader)) {
             final StandardProcessContext processContext = new StandardProcessContext(
                     existingNode, controllerServiceProvider, encryptor, getStateManagerProvider().getStateManager(id), variableRegistry);
             ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnRemoved.class, existingNode.getProcessor(), processContext);
+        } finally {
+            ExtensionManager.closeURLClassLoader(id, existingInstanceClassLoader);
         }
 
         // set the new processor in the existing node
@@ -1701,7 +1708,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
             //
             for (final ControllerServiceDTO controllerServiceDTO : dto.getControllerServices()) {
                 final BundleCoordinate bundleCoordinate = BundleUtils.getBundle(controllerServiceDTO.getType(), controllerServiceDTO.getBundle());
-                final ControllerServiceNode serviceNode = createControllerService(controllerServiceDTO.getType(), controllerServiceDTO.getId(), bundleCoordinate, true);
+                final ControllerServiceNode serviceNode = createControllerService(controllerServiceDTO.getType(), controllerServiceDTO.getId(), bundleCoordinate, Collections.emptySet(),true);
 
                 serviceNode.setAnnotationData(controllerServiceDTO.getAnnotationData());
                 serviceNode.setComments(controllerServiceDTO.getComments());
@@ -2942,12 +2949,13 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
     }
 
     @Override
-    public ReportingTaskNode createReportingTask(final String type, final String id, final BundleCoordinate bundleCoordinate,final boolean firstTimeAdded) throws ReportingTaskInstantiationException {
-        return createReportingTask(type, id, bundleCoordinate, firstTimeAdded, true);
+    public ReportingTaskNode createReportingTask(final String type, final String id, final BundleCoordinate bundleCoordinate,final boolean firstTimeAdded)
+            throws ReportingTaskInstantiationException {
+        return createReportingTask(type, id, bundleCoordinate, Collections.emptySet(), firstTimeAdded, true);
     }
 
-    public ReportingTaskNode createReportingTask(final String type, final String id, final BundleCoordinate bundleCoordinate, final boolean firstTimeAdded, final boolean register)
-            throws ReportingTaskInstantiationException {
+    public ReportingTaskNode createReportingTask(final String type, final String id, final BundleCoordinate bundleCoordinate, final Set<URL> additionalUrls,
+                                                 final boolean firstTimeAdded, final boolean register) throws ReportingTaskInstantiationException {
         if (type == null || id == null || bundleCoordinate == null) {
             throw new NullPointerException();
         }
@@ -2955,7 +2963,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
         LoggableComponent<ReportingTask> task = null;
         boolean creationSuccessful = true;
         try {
-            task = instantiateReportingTask(type, id, bundleCoordinate);
+            task = instantiateReportingTask(type, id, bundleCoordinate, additionalUrls);
         } catch (final Exception e) {
             LOG.error("Could not create Reporting Task of type " + type + " for ID " + id + "; creating \"Ghost\" implementation", e);
             final GhostReportingTask ghostTask = new GhostReportingTask();
@@ -2968,12 +2976,12 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
         final ValidationContextFactory validationContextFactory = new StandardValidationContextFactory(controllerServiceProvider, variableRegistry);
         final ReportingTaskNode taskNode;
         if (creationSuccessful) {
-            taskNode = new StandardReportingTaskNode(task, id, this, processScheduler, validationContextFactory, variableRegistry);
+            taskNode = new StandardReportingTaskNode(task, id, this, processScheduler, validationContextFactory, variableRegistry, this);
         } else {
             final String simpleClassName = type.contains(".") ? StringUtils.substringAfterLast(type, ".") : type;
             final String componentType = "(Missing) " + simpleClassName;
 
-            taskNode = new StandardReportingTaskNode(task, id, this, processScheduler, validationContextFactory, componentType, type, variableRegistry, true);
+            taskNode = new StandardReportingTaskNode(task, id, this, processScheduler, validationContextFactory, componentType, type, variableRegistry, this, true);
         }
 
         taskNode.setName(taskNode.getReportingTask().getClass().getSimpleName());
@@ -3008,7 +3016,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
         return taskNode;
     }
 
-    private LoggableComponent<ReportingTask> instantiateReportingTask(final String type, final String id, final BundleCoordinate bundleCoordinate)
+    private LoggableComponent<ReportingTask> instantiateReportingTask(final String type, final String id, final BundleCoordinate bundleCoordinate, final Set<URL> additionalUrls)
             throws ReportingTaskInstantiationException {
 
         final ClassLoader ctxClassLoader = Thread.currentThread().getContextClassLoader();
@@ -3018,7 +3026,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
                 throw new IllegalStateException("Unable to find bundle for coordinate " + bundleCoordinate.getCoordinate());
             }
 
-            final ClassLoader detectedClassLoader = ExtensionManager.createInstanceClassLoader(type, id, reportingTaskBundle);
+            final ClassLoader detectedClassLoader = ExtensionManager.createInstanceClassLoader(type, id, reportingTaskBundle, additionalUrls);
             final Class<?> rawClass = Class.forName(type, false, detectedClassLoader);
             Thread.currentThread().setContextClassLoader(detectedClassLoader);
 
@@ -3039,7 +3047,8 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
     }
 
     @Override
-    public void changeReportingTaskType(final ReportingTaskNode existingNode, final String newType, final BundleCoordinate bundleCoordinate) throws ReportingTaskInstantiationException {
+    public void reload(final ReportingTaskNode existingNode, final String newType, final BundleCoordinate bundleCoordinate, final Set<URL> additionalUrls)
+            throws ReportingTaskInstantiationException {
         if (existingNode == null) {
             throw new IllegalStateException("Existing ReportingTaskNode cannot be null");
         }
@@ -3052,11 +3061,13 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
 
         // set firstTimeAdded to true so lifecycle annotations get fired, but don't register this node
         // attempt the creation to make sure it works before firing the OnRemoved methods below
-        final ReportingTaskNode newNode = createReportingTask(newType, id, bundleCoordinate, true, false);
+        final ReportingTaskNode newNode = createReportingTask(newType, id, bundleCoordinate, additionalUrls, true, false);
 
         // call OnRemoved for the existing reporting task using the previous instance class loader
         try (final NarCloseable x = NarCloseable.withComponentNarLoader(existingInstanceClassLoader)) {
             ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnRemoved.class, existingNode.getReportingTask(), existingNode.getConfigurationContext());
+        } finally {
+            ExtensionManager.closeURLClassLoader(id, existingInstanceClassLoader);
         }
 
         // set the new reporting task into the existing node
@@ -3117,7 +3128,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
         }
 
         reportingTasks.remove(reportingTaskNode.getIdentifier());
-        ExtensionManager.removeInstanceClassLoaderIfExists(reportingTaskNode.getIdentifier());
+        ExtensionManager.removeInstanceClassLoader(reportingTaskNode.getIdentifier());
     }
 
     @Override
@@ -3126,8 +3137,8 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
     }
 
     @Override
-    public ControllerServiceNode createControllerService(final String type, final String id, final BundleCoordinate bundleCoordinate, final boolean firstTimeAdded) {
-        final ControllerServiceNode serviceNode = controllerServiceProvider.createControllerService(type, id, bundleCoordinate, firstTimeAdded);
+    public ControllerServiceNode createControllerService(final String type, final String id, final BundleCoordinate bundleCoordinate, final Set<URL> additionalUrls, final boolean firstTimeAdded) {
+        final ControllerServiceNode serviceNode = controllerServiceProvider.createControllerService(type, id, bundleCoordinate, additionalUrls, firstTimeAdded);
 
         // Register log observer to provide bulletins when reporting task logs anything at WARN level or above
         final LogRepository logRepository = LogRepositoryFactory.getRepository(id);
@@ -3145,7 +3156,8 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
         return serviceNode;
     }
 
-    public void changeControllerServiceType(final ControllerServiceNode existingNode, final String newType, final BundleCoordinate bundleCoordinate)
+    @Override
+    public void reload(final ControllerServiceNode existingNode, final String newType, final BundleCoordinate bundleCoordinate, final Set<URL> additionalUrls)
             throws ControllerServiceInstantiationException {
         if (existingNode == null) {
             throw new IllegalStateException("Existing ControllerServiceNode cannot be null");
@@ -3159,12 +3171,14 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
 
         // create a new node with firstTimeAdded as true so lifecycle methods get called
         // attempt the creation to make sure it works before firing the OnRemoved methods below
-        final ControllerServiceNode newNode = controllerServiceProvider.createControllerService(newType, id, bundleCoordinate, true);
+        final ControllerServiceNode newNode = controllerServiceProvider.createControllerService(newType, id, bundleCoordinate, additionalUrls, true);
 
         // call OnRemoved for the existing service using the previous instance class loader
         try (final NarCloseable x = NarCloseable.withComponentNarLoader(existingInstanceClassLoader)) {
             final ConfigurationContext configurationContext = new StandardConfigurationContext(existingNode, controllerServiceProvider, null, variableRegistry);
             ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnRemoved.class, existingNode.getControllerServiceImplementation(), configurationContext);
+        } finally {
+            ExtensionManager.closeURLClassLoader(id, existingInstanceClassLoader);
         }
 
         // take the invocation handler that was created for new proxy and is set to look at the new node,
@@ -3313,7 +3327,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
         rootControllerServices.remove(service.getIdentifier());
         getStateManagerProvider().onComponentRemoved(service.getIdentifier());
 
-        ExtensionManager.removeInstanceClassLoaderIfExists(service.getIdentifier());
+        ExtensionManager.removeInstanceClassLoader(service.getIdentifier());
 
         LOG.info("{} removed from Flow Controller", service, this);
     }

http://git-wip-us.apache.org/repos/asf/nifi/blob/556f309d/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardProcessorNode.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardProcessorNode.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardProcessorNode.java
index afc94d8..281c695 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardProcessorNode.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardProcessorNode.java
@@ -40,6 +40,7 @@ import org.apache.nifi.connectable.Connectable;
 import org.apache.nifi.connectable.ConnectableType;
 import org.apache.nifi.connectable.Connection;
 import org.apache.nifi.connectable.Position;
+import org.apache.nifi.controller.exception.ProcessorInstantiationException;
 import org.apache.nifi.controller.scheduling.ScheduleState;
 import org.apache.nifi.controller.scheduling.SchedulingAgent;
 import org.apache.nifi.controller.service.ControllerServiceNode;
@@ -66,6 +67,7 @@ import org.slf4j.LoggerFactory;
 import org.springframework.util.Assert;
 
 import java.lang.reflect.InvocationTargetException;
+import java.net.URL;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
@@ -132,19 +134,19 @@ public class StandardProcessorNode extends ProcessorNode implements Connectable
     public StandardProcessorNode(final LoggableComponent<Processor> processor, final String uuid,
                                  final ValidationContextFactory validationContextFactory, final ProcessScheduler scheduler,
                                  final ControllerServiceProvider controllerServiceProvider, final NiFiProperties nifiProperties,
-                                 final VariableRegistry variableRegistry) {
+                                 final VariableRegistry variableRegistry, final ReloadComponent reloadComponent) {
 
         this(processor, uuid, validationContextFactory, scheduler, controllerServiceProvider,
-            processor.getComponent().getClass().getSimpleName(), processor.getComponent().getClass().getCanonicalName(), nifiProperties, variableRegistry, false);
+            processor.getComponent().getClass().getSimpleName(), processor.getComponent().getClass().getCanonicalName(), nifiProperties, variableRegistry, reloadComponent, false);
     }
 
     public StandardProcessorNode(final LoggableComponent<Processor> processor, final String uuid,
                                  final ValidationContextFactory validationContextFactory, final ProcessScheduler scheduler,
                                  final ControllerServiceProvider controllerServiceProvider,
                                  final String componentType, final String componentCanonicalClass, final NiFiProperties nifiProperties,
-                                 final VariableRegistry variableRegistry, final boolean isExtensionMissing) {
+                                 final VariableRegistry variableRegistry, final ReloadComponent reloadComponent, final boolean isExtensionMissing) {
 
-        super(uuid, validationContextFactory, controllerServiceProvider, componentType, componentCanonicalClass, variableRegistry, isExtensionMissing);
+        super(uuid, validationContextFactory, controllerServiceProvider, componentType, componentCanonicalClass, variableRegistry, reloadComponent, isExtensionMissing);
 
         final ProcessorDetails processorDetails = new ProcessorDetails(processor);
         this.processorRef = new AtomicReference<>(processorDetails);
@@ -867,6 +869,15 @@ public class StandardProcessorNode extends ProcessorNode implements Connectable
         processorRef.set(processorDetails);
     }
 
+    @Override
+    public synchronized void reload(final Set<URL> additionalUrls) throws ProcessorInstantiationException {
+        if (isRunning()) {
+            throw new IllegalStateException("Cannot reload Processor while the Processor is running");
+        }
+
+        getReloadComponent().reload(this, getCanonicalClassName(), getBundleCoordinate(), additionalUrls);
+    }
+
     /**
      * @return the Set of destination processors for all relationships excluding
      *         any destinations that are this processor itself (self-loops)

http://git-wip-us.apache.org/repos/asf/nifi/blob/556f309d/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/reporting/AbstractReportingTaskNode.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/reporting/AbstractReportingTaskNode.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/reporting/AbstractReportingTaskNode.java
index aeb1d07..1d5d7b9 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/reporting/AbstractReportingTaskNode.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/reporting/AbstractReportingTaskNode.java
@@ -21,6 +21,7 @@ import org.apache.nifi.bundle.BundleCoordinate;
 import org.apache.nifi.components.ConfigurableComponent;
 import org.apache.nifi.components.ValidationResult;
 import org.apache.nifi.controller.AbstractConfiguredComponent;
+import org.apache.nifi.controller.ReloadComponent;
 import org.apache.nifi.controller.ConfigurationContext;
 import org.apache.nifi.controller.ControllerServiceLookup;
 import org.apache.nifi.controller.LoggableComponent;
@@ -37,6 +38,7 @@ import org.apache.nifi.reporting.ReportingTask;
 import org.apache.nifi.scheduling.SchedulingStrategy;
 import org.apache.nifi.util.FormatUtils;
 
+import java.net.URL;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.HashSet;
@@ -64,19 +66,21 @@ public abstract class AbstractReportingTaskNode extends AbstractConfiguredCompon
 
     public AbstractReportingTaskNode(final LoggableComponent<ReportingTask> reportingTask, final String id,
                                      final ControllerServiceProvider controllerServiceProvider, final ProcessScheduler processScheduler,
-                                     final ValidationContextFactory validationContextFactory, final VariableRegistry variableRegistry) {
+                                     final ValidationContextFactory validationContextFactory, final VariableRegistry variableRegistry,
+                                     final ReloadComponent reloadComponent) {
 
         this(reportingTask, id, controllerServiceProvider, processScheduler, validationContextFactory,
-            reportingTask.getComponent().getClass().getSimpleName(), reportingTask.getComponent().getClass().getCanonicalName(),variableRegistry, false);
+                reportingTask.getComponent().getClass().getSimpleName(), reportingTask.getComponent().getClass().getCanonicalName(),
+                variableRegistry, reloadComponent, false);
     }
 
 
     public AbstractReportingTaskNode(final LoggableComponent<ReportingTask> reportingTask, final String id, final ControllerServiceProvider controllerServiceProvider,
                                      final ProcessScheduler processScheduler, final ValidationContextFactory validationContextFactory,
                                      final String componentType, final String componentCanonicalClass, final VariableRegistry variableRegistry,
-                                     final boolean isExtensionMissing) {
+                                     final ReloadComponent reloadComponent, final boolean isExtensionMissing) {
 
-        super(id, validationContextFactory, controllerServiceProvider, componentType, componentCanonicalClass, variableRegistry, isExtensionMissing);
+        super(id, validationContextFactory, controllerServiceProvider, componentType, componentCanonicalClass, variableRegistry, reloadComponent, isExtensionMissing);
         this.reportingTaskRef = new AtomicReference<>(new ReportingTaskDetails(reportingTask));
         this.processScheduler = processScheduler;
         this.serviceLookup = controllerServiceProvider;
@@ -153,6 +157,15 @@ public abstract class AbstractReportingTaskNode extends AbstractConfiguredCompon
     }
 
     @Override
+    public void reload(final Set<URL> additionalUrls) throws ReportingTaskInstantiationException {
+        if (isRunning()) {
+            throw new IllegalStateException("Cannot reload Reporting Task while Reporting Task is running");
+        }
+
+        getReloadComponent().reload(this, getCanonicalClassName(), getBundleCoordinate(), additionalUrls);
+    }
+
+    @Override
     public boolean isRunning() {
         return processScheduler.isScheduled(this) || processScheduler.getActiveThreadCount(this) > 0;
     }

http://git-wip-us.apache.org/repos/asf/nifi/blob/556f309d/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/reporting/StandardReportingTaskNode.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/reporting/StandardReportingTaskNode.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/reporting/StandardReportingTaskNode.java
index edf1b67..3dbfce1 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/reporting/StandardReportingTaskNode.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/reporting/StandardReportingTaskNode.java
@@ -21,6 +21,7 @@ import org.apache.nifi.authorization.Resource;
 import org.apache.nifi.authorization.resource.Authorizable;
 import org.apache.nifi.authorization.resource.ResourceFactory;
 import org.apache.nifi.authorization.resource.ResourceType;
+import org.apache.nifi.controller.ReloadComponent;
 import org.apache.nifi.controller.FlowController;
 import org.apache.nifi.controller.LoggableComponent;
 import org.apache.nifi.controller.ProcessScheduler;
@@ -36,16 +37,17 @@ public class StandardReportingTaskNode extends AbstractReportingTaskNode impleme
 
     public StandardReportingTaskNode(final LoggableComponent<ReportingTask> reportingTask, final String id, final FlowController controller,
                                      final ProcessScheduler processScheduler, final ValidationContextFactory validationContextFactory,
-                                     final VariableRegistry variableRegistry) {
-        super(reportingTask, id, controller, processScheduler, validationContextFactory, variableRegistry);
+                                     final VariableRegistry variableRegistry, final ReloadComponent reloadComponent) {
+        super(reportingTask, id, controller, processScheduler, validationContextFactory, variableRegistry, reloadComponent);
         this.flowController = controller;
     }
 
     public StandardReportingTaskNode(final LoggableComponent<ReportingTask> reportingTask, final String id, final FlowController controller,
                                      final ProcessScheduler processScheduler, final ValidationContextFactory validationContextFactory,
                                      final String componentType, final String canonicalClassName, final VariableRegistry variableRegistry,
-                                     final boolean isExtensionMissing) {
-        super(reportingTask, id, controller, processScheduler, validationContextFactory, componentType, canonicalClassName,variableRegistry, isExtensionMissing);
+                                     final ReloadComponent reloadComponent, final boolean isExtensionMissing) {
+        super(reportingTask, id, controller, processScheduler, validationContextFactory, componentType, canonicalClassName,
+                variableRegistry, reloadComponent, isExtensionMissing);
         this.flowController = controller;
     }
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/556f309d/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/ControllerServiceLoader.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/ControllerServiceLoader.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/ControllerServiceLoader.java
index 520d3fb..3faffd7 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/ControllerServiceLoader.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/ControllerServiceLoader.java
@@ -43,6 +43,7 @@ import java.io.InputStream;
 import java.nio.charset.StandardCharsets;
 import java.util.ArrayList;
 import java.util.Collection;
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
@@ -163,7 +164,8 @@ public class ControllerServiceLoader {
         // create a new id for the clone seeded from the original id so that it is consistent in a cluster
         final UUID id = UUID.nameUUIDFromBytes(controllerService.getIdentifier().getBytes(StandardCharsets.UTF_8));
 
-        final ControllerServiceNode clone = provider.createControllerService(controllerService.getCanonicalClassName(), id.toString(), controllerService.getBundleCoordinate(), false);
+        final ControllerServiceNode clone = provider.createControllerService(controllerService.getCanonicalClassName(), id.toString(),
+                controllerService.getBundleCoordinate(), Collections.emptySet(), false);
         clone.setName(controllerService.getName());
         clone.setComments(controllerService.getComments());
 
@@ -193,7 +195,7 @@ public class ControllerServiceLoader {
             }
         }
 
-        final ControllerServiceNode node = provider.createControllerService(dto.getType(), dto.getId(), coordinate, false);
+        final ControllerServiceNode node = provider.createControllerService(dto.getType(), dto.getId(), coordinate, Collections.emptySet(), false);
         node.setName(dto.getName());
         node.setComments(dto.getComments());
         return node;

http://git-wip-us.apache.org/repos/asf/nifi/blob/556f309d/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceNode.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceNode.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceNode.java
index a543040..4ee65ec 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceNode.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceNode.java
@@ -29,11 +29,13 @@ import org.apache.nifi.components.ConfigurableComponent;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.ValidationResult;
 import org.apache.nifi.controller.AbstractConfiguredComponent;
+import org.apache.nifi.controller.ReloadComponent;
 import org.apache.nifi.controller.ConfigurationContext;
 import org.apache.nifi.controller.ConfiguredComponent;
 import org.apache.nifi.controller.ControllerService;
 import org.apache.nifi.controller.LoggableComponent;
 import org.apache.nifi.controller.ValidationContextFactory;
+import org.apache.nifi.controller.exception.ControllerServiceInstantiationException;
 import org.apache.nifi.groups.ProcessGroup;
 import org.apache.nifi.logging.ComponentLog;
 import org.apache.nifi.nar.NarCloseable;
@@ -44,6 +46,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.lang.reflect.InvocationTargetException;
+import java.net.URL;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
@@ -79,18 +82,18 @@ public class StandardControllerServiceNode extends AbstractConfiguredComponent i
 
     public StandardControllerServiceNode(final LoggableComponent<ControllerService> implementation, final LoggableComponent<ControllerService> proxiedControllerService,
                                          final ControllerServiceInvocationHandler invocationHandler, final String id, final ValidationContextFactory validationContextFactory,
-                                         final ControllerServiceProvider serviceProvider, final VariableRegistry variableRegistry) {
+                                         final ControllerServiceProvider serviceProvider, final VariableRegistry variableRegistry, final ReloadComponent reloadComponent) {
 
         this(implementation, proxiedControllerService, invocationHandler, id, validationContextFactory, serviceProvider,
-            implementation.getComponent().getClass().getSimpleName(), implementation.getComponent().getClass().getCanonicalName(), variableRegistry, false);
+            implementation.getComponent().getClass().getSimpleName(), implementation.getComponent().getClass().getCanonicalName(), variableRegistry, reloadComponent, false);
     }
 
     public StandardControllerServiceNode(final LoggableComponent<ControllerService> implementation, final LoggableComponent<ControllerService> proxiedControllerService,
                                          final ControllerServiceInvocationHandler invocationHandler, final String id, final ValidationContextFactory validationContextFactory,
                                          final ControllerServiceProvider serviceProvider, final String componentType, final String componentCanonicalClass,
-                                         final VariableRegistry variableRegistry, final boolean isExtensionMissing) {
+                                         final VariableRegistry variableRegistry, final ReloadComponent reloadComponent, final boolean isExtensionMissing) {
 
-        super(id, validationContextFactory, serviceProvider, componentType, componentCanonicalClass, variableRegistry, isExtensionMissing);
+        super(id, validationContextFactory, serviceProvider, componentType, componentCanonicalClass, variableRegistry, reloadComponent, isExtensionMissing);
         this.serviceProvider = serviceProvider;
         this.active = new AtomicBoolean();
         setControllerServiceAndProxy(implementation, proxiedControllerService, invocationHandler);
@@ -172,6 +175,17 @@ public class StandardControllerServiceNode extends AbstractConfiguredComponent i
     }
 
     @Override
+    public void reload(final Set<URL> additionalUrls) throws ControllerServiceInstantiationException {
+        synchronized (this.active) {
+            if (isActive()) {
+                throw new IllegalStateException("Cannot reload Controller Service while service is active");
+            }
+
+            getReloadComponent().reload(this, getCanonicalClassName(), getBundleCoordinate(), additionalUrls);
+        }
+    }
+
+    @Override
     public ProcessGroup getProcessGroup() {
         readLock.lock();
         try {

http://git-wip-us.apache.org/repos/asf/nifi/blob/556f309d/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceProvider.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceProvider.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceProvider.java
index cadc8e7..ab70e75 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceProvider.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceProvider.java
@@ -53,6 +53,7 @@ import org.slf4j.LoggerFactory;
 
 import java.lang.reflect.Method;
 import java.lang.reflect.Proxy;
+import java.net.URL;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
@@ -91,7 +92,7 @@ public class StandardControllerServiceProvider implements ControllerServiceProvi
     }
 
     @Override
-    public ControllerServiceNode createControllerService(final String type, final String id, final BundleCoordinate bundleCoordinate, final boolean firstTimeAdded) {
+    public ControllerServiceNode createControllerService(final String type, final String id, final BundleCoordinate bundleCoordinate, final Set<URL> additionalUrls, final boolean firstTimeAdded) {
         if (type == null || id == null || bundleCoordinate == null) {
             throw new NullPointerException();
         }
@@ -106,7 +107,7 @@ public class StandardControllerServiceProvider implements ControllerServiceProvi
                     throw new ControllerServiceInstantiationException("Unable to find bundle for coordinate " + bundleCoordinate.getCoordinate());
                 }
 
-                cl = ExtensionManager.createInstanceClassLoader(type, id, csBundle);
+                cl = ExtensionManager.createInstanceClassLoader(type, id, csBundle, additionalUrls);
                 Thread.currentThread().setContextClassLoader(cl);
                 rawClass = Class.forName(type, false, cl);
             } catch (final Exception e) {
@@ -141,7 +142,7 @@ public class StandardControllerServiceProvider implements ControllerServiceProvi
             final LoggableComponent<ControllerService> proxiedLoggableComponent = new LoggableComponent<>(proxiedService, bundleCoordinate, serviceLogger);
 
             final ControllerServiceNode serviceNode = new StandardControllerServiceNode(originalLoggableComponent, proxiedLoggableComponent, invocationHandler,
-                    id, validationContextFactory, this, variableRegistry);
+                    id, validationContextFactory, this, variableRegistry, flowController);
             serviceNode.setName(rawClass.getSimpleName());
 
             invocationHandler.setServiceNode(serviceNode);
@@ -217,7 +218,7 @@ public class StandardControllerServiceProvider implements ControllerServiceProvi
         final LoggableComponent<ControllerService> proxiedLoggableComponent = new LoggableComponent<>(proxiedService, bundleCoordinate, null);
 
         final ControllerServiceNode serviceNode = new StandardControllerServiceNode(proxiedLoggableComponent, proxiedLoggableComponent, invocationHandler, id,
-                new StandardValidationContextFactory(this, variableRegistry), this, componentType, type, variableRegistry, true);
+                new StandardValidationContextFactory(this, variableRegistry), this, componentType, type, variableRegistry, flowController, true);
         return serviceNode;
     }
 
@@ -537,7 +538,7 @@ public class StandardControllerServiceProvider implements ControllerServiceProvi
         }
 
         group.removeControllerService(serviceNode);
-        ExtensionManager.removeInstanceClassLoaderIfExists(serviceNode.getIdentifier());
+        ExtensionManager.removeInstanceClassLoader(serviceNode.getIdentifier());
     }
 
     @Override


[4/7] nifi git commit: NIFI-3520 Refactoring instance class loading - Fixing FlowController to use appropriate class loader when instantiating processor - Updating ExtensionManager to leverage new flag in MANIFEST from NAR plugin - Adding ReloadComponent

Posted by mc...@apache.org.
http://git-wip-us.apache.org/repos/asf/nifi/blob/556f309d/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/nar/NarThreadContextClassLoader.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/nar/NarThreadContextClassLoader.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/nar/NarThreadContextClassLoader.java
new file mode 100644
index 0000000..e7faa02
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/nar/NarThreadContextClassLoader.java
@@ -0,0 +1,235 @@
+/*
+ * 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 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;
+import org.apache.nifi.controller.repository.FlowFileSwapManager;
+import org.apache.nifi.controller.status.history.ComponentStatusRepository;
+import org.apache.nifi.flowfile.FlowFilePrioritizer;
+import org.apache.nifi.processor.Processor;
+import org.apache.nifi.processor.io.InputStreamCallback;
+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;
+import java.lang.reflect.Constructor;
+import java.lang.reflect.InvocationTargetException;
+import java.net.URL;
+import java.net.URLClassLoader;
+import java.util.ArrayList;
+import java.util.Enumeration;
+import java.util.List;
+
+/**
+ * THREAD SAFE
+ */
+public class NarThreadContextClassLoader extends URLClassLoader {
+
+    static final ContextSecurityManager contextSecurityManager = new ContextSecurityManager();
+    private final ClassLoader forward = ClassLoader.getSystemClassLoader();
+    private static final List<Class<?>> narSpecificClasses = new ArrayList<>();
+
+    static {
+        narSpecificClasses.add(Processor.class);
+        narSpecificClasses.add(FlowFilePrioritizer.class);
+        narSpecificClasses.add(ReportingTask.class);
+        narSpecificClasses.add(Validator.class);
+        narSpecificClasses.add(InputStreamCallback.class);
+        narSpecificClasses.add(OutputStreamCallback.class);
+        narSpecificClasses.add(StreamCallback.class);
+        narSpecificClasses.add(ControllerService.class);
+        narSpecificClasses.add(Authorizer.class);
+        narSpecificClasses.add(LoginIdentityProvider.class);
+        narSpecificClasses.add(ProvenanceRepository.class);
+        narSpecificClasses.add(ComponentStatusRepository.class);
+        narSpecificClasses.add(FlowFileRepository.class);
+        narSpecificClasses.add(FlowFileSwapManager.class);
+        narSpecificClasses.add(ContentRepository.class);
+        narSpecificClasses.add(StateProvider.class);
+    }
+
+    private NarThreadContextClassLoader() {
+        super(new URL[0]);
+    }
+
+    @Override
+    public void clearAssertionStatus() {
+        lookupClassLoader().clearAssertionStatus();
+    }
+
+    @Override
+    public URL getResource(String name) {
+        return lookupClassLoader().getResource(name);
+    }
+
+    @Override
+    public InputStream getResourceAsStream(String name) {
+        return lookupClassLoader().getResourceAsStream(name);
+    }
+
+    @Override
+    public Enumeration<URL> getResources(String name) throws IOException {
+        return lookupClassLoader().getResources(name);
+    }
+
+    @Override
+    public Class<?> loadClass(String name) throws ClassNotFoundException {
+        return lookupClassLoader().loadClass(name);
+    }
+
+    @Override
+    public void setClassAssertionStatus(String className, boolean enabled) {
+        lookupClassLoader().setClassAssertionStatus(className, enabled);
+    }
+
+    @Override
+    public void setDefaultAssertionStatus(boolean enabled) {
+        lookupClassLoader().setDefaultAssertionStatus(enabled);
+    }
+
+    @Override
+    public void setPackageAssertionStatus(String packageName, boolean enabled) {
+        lookupClassLoader().setPackageAssertionStatus(packageName, enabled);
+    }
+
+    private ClassLoader lookupClassLoader() {
+        final Class<?>[] classStack = contextSecurityManager.getExecutionStack();
+
+        for (Class<?> currentClass : classStack) {
+            final Class<?> narClass = findNarClass(currentClass);
+            if (narClass != null) {
+                final ClassLoader desiredClassLoader = narClass.getClassLoader();
+
+                // When new Threads are created, the new Thread inherits the ClassLoaderContext of
+                // the caller. However, the call stack of that new Thread may not trace back to any NiFi-specific
+                // code. Therefore, the NarThreadContextClassLoader will be unable to find the appropriate NAR
+                // ClassLoader. As a result, we want to set the ContextClassLoader to the NAR ClassLoader that
+                // contains the class or resource that we are looking for.
+                // This locks the current Thread into the appropriate NAR ClassLoader Context. The framework will change
+                // the ContextClassLoader back to the NarThreadContextClassLoader as appropriate via the
+                // {@link FlowEngine.beforeExecute(Thread, Runnable)} and
+                // {@link FlowEngine.afterExecute(Thread, Runnable)} methods.
+                if (desiredClassLoader instanceof NarClassLoader) {
+                    Thread.currentThread().setContextClassLoader(desiredClassLoader);
+                }
+                return desiredClassLoader;
+            }
+        }
+        return forward;
+    }
+
+    private Class<?> findNarClass(final Class<?> cls) {
+        for (final Class<?> narClass : narSpecificClasses) {
+            if (narClass.isAssignableFrom(cls)) {
+                return cls;
+            } else if (cls.getEnclosingClass() != null) {
+                return findNarClass(cls.getEnclosingClass());
+            }
+        }
+
+        return null;
+    }
+
+    private static class SingletonHolder {
+
+        public static final NarThreadContextClassLoader instance = new NarThreadContextClassLoader();
+    }
+
+    public static NarThreadContextClassLoader getInstance() {
+        return SingletonHolder.instance;
+    }
+
+    static class ContextSecurityManager extends SecurityManager {
+
+        Class<?>[] getExecutionStack() {
+            return getClassContext();
+        }
+    }
+
+    /**
+     * Constructs an instance of the given type using either default no args
+     * constructor or a constructor which takes a NiFiProperties object
+     * (preferred).
+     *
+     * @param <T> the type to create an instance for
+     * @param implementationClassName the implementation class name
+     * @param typeDefinition the type definition
+     * @param nifiProperties the NiFiProperties instance
+     * @return constructed instance
+     * @throws InstantiationException if there is an error instantiating the class
+     * @throws IllegalAccessException if there is an error accessing the type
+     * @throws ClassNotFoundException if the class cannot be found
+     */
+    public static <T> T createInstance(final String implementationClassName, final Class<T> typeDefinition, final NiFiProperties nifiProperties)
+            throws InstantiationException, IllegalAccessException, ClassNotFoundException {
+        final ClassLoader originalClassLoader = Thread.currentThread().getContextClassLoader();
+        Thread.currentThread().setContextClassLoader(NarThreadContextClassLoader.getInstance());
+        try {
+            final List<Bundle> bundles = ExtensionManager.getBundles(implementationClassName);
+            if (bundles.size() == 0) {
+                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);
+            if(nifiProperties == null){
+                return typeDefinition.cast(desiredClass.newInstance());
+            }
+            Constructor<?> constructor = null;
+
+            try {
+                constructor = desiredClass.getConstructor(NiFiProperties.class);
+            } catch (NoSuchMethodException nsme) {
+                try {
+                    constructor = desiredClass.getConstructor();
+                } catch (NoSuchMethodException nsme2) {
+                    throw new IllegalStateException("Failed to find constructor which takes NiFiProperties as argument as well as the default constructor on "
+                            + desiredClass.getName(), nsme2);
+                }
+            }
+            try {
+                if (constructor.getParameterTypes().length == 0) {
+                    return typeDefinition.cast(constructor.newInstance());
+                } else {
+                    return typeDefinition.cast(constructor.newInstance(nifiProperties));
+                }
+            } catch (InvocationTargetException ite) {
+                throw new IllegalStateException("Failed to instantiate a component due to (see target exception)", ite);
+            }
+        } finally {
+            Thread.currentThread().setContextClassLoader(originalClassLoader);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/556f309d/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/test/java/org/apache/nifi/nar/NarThreadContextClassLoaderTest.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/test/java/org/apache/nifi/nar/NarThreadContextClassLoaderTest.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/test/java/org/apache/nifi/nar/NarThreadContextClassLoaderTest.java
new file mode 100644
index 0000000..39014bc
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/test/java/org/apache/nifi/nar/NarThreadContextClassLoaderTest.java
@@ -0,0 +1,99 @@
+/*
+ * 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.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;
+
+public class NarThreadContextClassLoaderTest {
+
+    @Test
+    public void validateWithPropertiesConstructor() throws Exception {
+        NiFiProperties properties = NiFiProperties.createBasicNiFiProperties("src/test/resources/nifi.properties", null);
+        Bundle systemBundle = SystemBundle.create(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)
+    public void validateWithPropertiesConstructorInstantiationFailure() throws Exception {
+        Map<String, String> additionalProperties = new HashMap<>();
+        additionalProperties.put("fail", "true");
+        NiFiProperties properties = NiFiProperties.createBasicNiFiProperties("src/test/resources/nifi.properties", additionalProperties);
+        Bundle systemBundle = SystemBundle.create(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 = SystemBundle.create(properties);
+        ExtensionManager.discoverExtensions(systemBundle, Collections.emptySet());
+        assertTrue(NarThreadContextClassLoader.createInstance(WithDefaultConstructor.class.getName(),
+                WithDefaultConstructor.class, properties) instanceof WithDefaultConstructor);
+    }
+
+    public static class WithPropertiesConstructor extends AbstractProcessor {
+        private NiFiProperties properties;
+
+        public 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 extends AbstractProcessor {
+        public WithDefaultConstructor() {
+
+        }
+
+        @Override
+        public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException {
+
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/556f309d/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/test/java/org/apache/nifi/nar/NarUnpackerTest.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/test/java/org/apache/nifi/nar/NarUnpackerTest.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/test/java/org/apache/nifi/nar/NarUnpackerTest.java
new file mode 100644
index 0000000..ac666ca
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/test/java/org/apache/nifi/nar/NarUnpackerTest.java
@@ -0,0 +1,183 @@
+/*
+ * 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 org.apache.nifi.util.NiFiProperties;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.io.File;
+import java.io.IOException;
+import java.net.URISyntaxException;
+import java.nio.file.FileVisitResult;
+import java.nio.file.Files;
+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 static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+
+public class NarUnpackerTest {
+
+    @BeforeClass
+    public static void copyResources() throws IOException {
+
+        final Path sourcePath = Paths.get("./src/test/resources");
+        final Path targetPath = Paths.get("./target");
+
+        Files.walkFileTree(sourcePath, new SimpleFileVisitor<Path>() {
+
+            @Override
+            public FileVisitResult preVisitDirectory(Path dir, BasicFileAttributes attrs)
+                    throws IOException {
+
+                Path relativeSource = sourcePath.relativize(dir);
+                Path target = targetPath.resolve(relativeSource);
+
+                Files.createDirectories(target);
+
+                return FileVisitResult.CONTINUE;
+
+            }
+
+            @Override
+            public FileVisitResult visitFile(Path file, BasicFileAttributes attrs)
+                    throws IOException {
+
+                Path relativeSource = sourcePath.relativize(file);
+                Path target = targetPath.resolve(relativeSource);
+
+                Files.copy(file, target, REPLACE_EXISTING);
+
+                return FileVisitResult.CONTINUE;
+            }
+        });
+    }
+
+    @Test
+    public void testUnpackNars() {
+
+        NiFiProperties properties = loadSpecifiedProperties("/NarUnpacker/conf/nifi.properties", Collections.EMPTY_MAP);
+
+        assertEquals("./target/NarUnpacker/lib/",
+                properties.getProperty("nifi.nar.library.directory"));
+        assertEquals("./target/NarUnpacker/lib2/",
+                properties.getProperty("nifi.nar.library.directory.alt"));
+
+        final ExtensionMapping extensionMapping = NarUnpacker.unpackNars(properties, SystemBundle.create(properties));
+
+        assertEquals(2, extensionMapping.getAllExtensionNames().size());
+
+        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();
+
+        Set<String> expectedNars = new HashSet<>();
+        expectedNars.add("dummy-one.nar-unpacked");
+        expectedNars.add("dummy-two.nar-unpacked");
+        assertEquals(expectedNars.size(), extensionFiles.length);
+
+        for (File extensionFile : extensionFiles) {
+            Assert.assertTrue(expectedNars.contains(extensionFile.getName()));
+        }
+    }
+
+    @Test
+    public void testUnpackNarsFromEmptyDir() throws IOException {
+
+        final File emptyDir = new File("./target/empty/dir");
+        emptyDir.delete();
+        emptyDir.deleteOnExit();
+        assertTrue(emptyDir.mkdirs());
+
+        final Map<String, String> others = new HashMap<>();
+        others.put("nifi.nar.library.directory.alt", emptyDir.toString());
+        NiFiProperties properties = loadSpecifiedProperties("/NarUnpacker/conf/nifi.properties", others);
+
+        final ExtensionMapping extensionMapping = NarUnpacker.unpackNars(properties, SystemBundle.create(properties));
+
+        assertEquals(1, extensionMapping.getAllExtensionNames().size());
+        assertTrue(extensionMapping.getAllExtensionNames().keySet().contains("org.apache.nifi.processors.dummy.one"));
+
+        final File extensionsWorkingDir = properties.getExtensionsWorkingDirectory();
+        File[] extensionFiles = extensionsWorkingDir.listFiles();
+
+        assertEquals(1, extensionFiles.length);
+        assertEquals("dummy-one.nar-unpacked", extensionFiles[0].getName());
+    }
+
+    @Test
+    public void testUnpackNarsFromNonExistantDir() {
+
+        final File nonExistantDir = new File("./target/this/dir/should/not/exist/");
+        nonExistantDir.delete();
+        nonExistantDir.deleteOnExit();
+
+        final Map<String, String> others = new HashMap<>();
+        others.put("nifi.nar.library.directory.alt", nonExistantDir.toString());
+        NiFiProperties properties = loadSpecifiedProperties("/NarUnpacker/conf/nifi.properties", others);
+
+        final ExtensionMapping extensionMapping = NarUnpacker.unpackNars(properties, SystemBundle.create(properties));
+
+        assertTrue(extensionMapping.getAllExtensionNames().keySet().contains("org.apache.nifi.processors.dummy.one"));
+
+        assertEquals(1, extensionMapping.getAllExtensionNames().size());
+
+        final File extensionsWorkingDir = properties.getExtensionsWorkingDirectory();
+        File[] extensionFiles = extensionsWorkingDir.listFiles();
+
+        assertEquals(1, extensionFiles.length);
+        assertEquals("dummy-one.nar-unpacked", extensionFiles[0].getName());
+    }
+
+    @Test
+    public void testUnpackNarsFromNonDir() throws IOException {
+
+        final File nonDir = new File("./target/file.txt");
+        nonDir.createNewFile();
+        nonDir.deleteOnExit();
+
+        final Map<String, String> others = new HashMap<>();
+        others.put("nifi.nar.library.directory.alt", nonDir.toString());
+        NiFiProperties properties = loadSpecifiedProperties("/NarUnpacker/conf/nifi.properties", others);
+
+        final ExtensionMapping extensionMapping = NarUnpacker.unpackNars(properties, SystemBundle.create(properties));
+
+        assertNull(extensionMapping);
+    }
+
+    private NiFiProperties loadSpecifiedProperties(final String propertiesFile, final Map<String, String> others) {
+        String filePath;
+        try {
+            filePath = NarUnpackerTest.class.getResource(propertiesFile).toURI().getPath();
+        } catch (URISyntaxException 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/556f309d/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-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-framework-nar-utils/src/test/resources/META-INF/services/org.apache.nifi.processor.Processor b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-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-framework-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/556f309d/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/test/resources/NarUnpacker/conf/nifi.properties
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/test/resources/NarUnpacker/conf/nifi.properties b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/test/resources/NarUnpacker/conf/nifi.properties
new file mode 100644
index 0000000..3a17e0d
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/test/resources/NarUnpacker/conf/nifi.properties
@@ -0,0 +1,125 @@
+# 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.
+
+# Core Properties #
+nifi.flow.configuration.file=./target/flow.xml.gz
+nifi.flow.configuration.archive.dir=./target/archive/
+nifi.flowcontroller.autoResumeState=true
+nifi.flowcontroller.graceful.shutdown.period=10 sec
+nifi.flowservice.writedelay.interval=2 sec
+nifi.administrative.yield.duration=30 sec
+
+nifi.reporting.task.configuration.file=./target/reporting-tasks.xml
+nifi.controller.service.configuration.file=./target/controller-services.xml
+nifi.templates.directory=./target/templates
+nifi.ui.banner.text=UI Banner Text
+nifi.ui.autorefresh.interval=30 sec
+nifi.nar.library.directory=./target/NarUnpacker/lib/
+nifi.nar.library.directory.alt=./target/NarUnpacker/lib2/
+
+nifi.nar.working.directory=./target/work/nar/
+nifi.documentation.working.directory=./target/work/docs/components
+
+# H2 Settings
+nifi.database.directory=./target/database_repository
+nifi.h2.url.append=;LOCK_TIMEOUT=25000;WRITE_DELAY=0;AUTO_SERVER=FALSE
+
+# FlowFile Repository
+nifi.flowfile.repository.directory=./target/test-repo
+nifi.flowfile.repository.partitions=1
+nifi.flowfile.repository.checkpoint.interval=2 mins
+nifi.queue.swap.threshold=20000
+nifi.swap.storage.directory=./target/test-repo/swap
+nifi.swap.in.period=5 sec
+nifi.swap.in.threads=1
+nifi.swap.out.period=5 sec
+nifi.swap.out.threads=4
+
+# Content Repository
+nifi.content.claim.max.appendable.size=10 MB
+nifi.content.claim.max.flow.files=100
+nifi.content.repository.directory.default=./target/content_repository
+
+# Provenance Repository Properties
+nifi.provenance.repository.storage.directory=./target/provenance_repository
+nifi.provenance.repository.max.storage.time=24 hours
+nifi.provenance.repository.max.storage.size=1 GB
+nifi.provenance.repository.rollover.time=30 secs
+nifi.provenance.repository.rollover.size=100 MB
+
+# Site to Site properties
+nifi.remote.input.socket.port=9990
+nifi.remote.input.secure=true
+
+# web properties #
+nifi.web.war.directory=./target/lib
+nifi.web.http.host=
+nifi.web.http.port=8080
+nifi.web.https.host=
+nifi.web.https.port=
+nifi.web.jetty.working.directory=./target/work/jetty
+
+# security properties #
+nifi.sensitive.props.key=key
+nifi.sensitive.props.algorithm=PBEWITHMD5AND256BITAES-CBC-OPENSSL
+nifi.sensitive.props.provider=BC
+
+nifi.security.keystore=
+nifi.security.keystoreType=
+nifi.security.keystorePasswd=
+nifi.security.keyPasswd=
+nifi.security.truststore=
+nifi.security.truststoreType=
+nifi.security.truststorePasswd=
+nifi.security.needClientAuth=
+nifi.security.user.authorizer=
+
+# cluster common properties (cluster manager and nodes must have same values) #
+nifi.cluster.protocol.heartbeat.interval=5 sec
+nifi.cluster.protocol.is.secure=false
+nifi.cluster.protocol.socket.timeout=30 sec
+nifi.cluster.protocol.connection.handshake.timeout=45 sec
+# if multicast is used, then nifi.cluster.protocol.multicast.xxx properties must be configured #
+nifi.cluster.protocol.use.multicast=false
+nifi.cluster.protocol.multicast.address=
+nifi.cluster.protocol.multicast.port=
+nifi.cluster.protocol.multicast.service.broadcast.delay=500 ms
+nifi.cluster.protocol.multicast.service.locator.attempts=3
+nifi.cluster.protocol.multicast.service.locator.attempts.delay=1 sec
+
+# cluster node properties (only configure for cluster nodes) #
+nifi.cluster.is.node=false
+nifi.cluster.node.address=
+nifi.cluster.node.protocol.port=
+nifi.cluster.node.protocol.threads=2
+# if multicast is not used, nifi.cluster.node.unicast.xxx must have same values as nifi.cluster.manager.xxx #
+nifi.cluster.node.unicast.manager.address=
+nifi.cluster.node.unicast.manager.protocol.port=
+nifi.cluster.node.unicast.manager.authority.provider.port=
+
+# cluster manager properties (only configure for cluster manager) #
+nifi.cluster.is.manager=false
+nifi.cluster.manager.address=
+nifi.cluster.manager.protocol.port=
+nifi.cluster.manager.authority.provider.port=
+nifi.cluster.manager.authority.provider.threads=10
+nifi.cluster.manager.node.firewall.file=
+nifi.cluster.manager.node.event.history.size=10
+nifi.cluster.manager.node.api.connection.timeout=30 sec
+nifi.cluster.manager.node.api.read.timeout=30 sec
+nifi.cluster.manager.node.api.request.threads=10
+nifi.cluster.manager.flow.retrieval.delay=5 sec
+nifi.cluster.manager.protocol.threads=10
+nifi.cluster.manager.safemode.duration=0 sec

http://git-wip-us.apache.org/repos/asf/nifi/blob/556f309d/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/test/resources/NarUnpacker/lib/dummy-one.nar
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/test/resources/NarUnpacker/lib/dummy-one.nar b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/test/resources/NarUnpacker/lib/dummy-one.nar
new file mode 100644
index 0000000..598b27f
Binary files /dev/null and b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/test/resources/NarUnpacker/lib/dummy-one.nar differ

http://git-wip-us.apache.org/repos/asf/nifi/blob/556f309d/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/test/resources/NarUnpacker/lib/nifi-framework-nar.nar
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/test/resources/NarUnpacker/lib/nifi-framework-nar.nar b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/test/resources/NarUnpacker/lib/nifi-framework-nar.nar
new file mode 100644
index 0000000..d2a8b96
Binary files /dev/null and b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/test/resources/NarUnpacker/lib/nifi-framework-nar.nar differ

http://git-wip-us.apache.org/repos/asf/nifi/blob/556f309d/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/test/resources/NarUnpacker/lib2/dummy-two.nar
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/test/resources/NarUnpacker/lib2/dummy-two.nar b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/test/resources/NarUnpacker/lib2/dummy-two.nar
new file mode 100644
index 0000000..a1021ba
Binary files /dev/null and b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/test/resources/NarUnpacker/lib2/dummy-two.nar differ

http://git-wip-us.apache.org/repos/asf/nifi/blob/556f309d/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/test/resources/nifi.properties
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/test/resources/nifi.properties b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/test/resources/nifi.properties
new file mode 100644
index 0000000..bbec968
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/test/resources/nifi.properties
@@ -0,0 +1,198 @@
+# 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.
+
+# Core Properties #
+nifi.flow.configuration.file=./conf/flow.xml.gz
+nifi.flow.configuration.archive.enabled=true
+nifi.flow.configuration.archive.dir=./conf/archive/
+nifi.flow.configuration.archive.max.time=30 days
+nifi.flow.configuration.archive.max.storage=500 MB
+nifi.flowcontroller.autoResumeState=true
+nifi.flowcontroller.graceful.shutdown.period=10 sec
+nifi.flowservice.writedelay.interval=500 ms
+nifi.administrative.yield.duration=30 sec
+# If a component has no work to do (is "bored"), how long should we wait before checking again for work?
+nifi.bored.yield.duration=10 millis
+
+nifi.authorizer.configuration.file=./conf/authorizers.xml
+nifi.login.identity.provider.configuration.file=./conf/login-identity-providers.xml
+nifi.templates.directory=./conf/templates
+nifi.ui.banner.text=
+nifi.ui.autorefresh.interval=30 sec
+nifi.nar.library.directory=./lib
+nifi.nar.working.directory=./target/work/nar/
+nifi.documentation.working.directory=./target/work/docs/components
+
+####################
+# State Management #
+####################
+nifi.state.management.configuration.file=./conf/state-management.xml
+# The ID of the local state provider
+nifi.state.management.provider.local=local-provider
+# The ID of the cluster-wide state provider. This will be ignored if NiFi is not clustered but must be populated if running in a cluster.
+nifi.state.management.provider.cluster=zk-provider
+# Specifies whether or not this instance of NiFi should run an embedded ZooKeeper server
+nifi.state.management.embedded.zookeeper.start=false
+# Properties file that provides the ZooKeeper properties to use if <nifi.state.management.embedded.zookeeper.start> is set to true
+nifi.state.management.embedded.zookeeper.properties=./conf/zookeeper.properties
+
+
+# H2 Settings
+nifi.database.directory=./database_repository
+nifi.h2.url.append=;LOCK_TIMEOUT=25000;WRITE_DELAY=0;AUTO_SERVER=FALSE
+
+# FlowFile Repository
+nifi.flowfile.repository.implementation=org.apache.nifi.controller.repository.WriteAheadFlowFileRepository
+nifi.flowfile.repository.directory=./flowfile_repository
+nifi.flowfile.repository.partitions=256
+nifi.flowfile.repository.checkpoint.interval=2 mins
+nifi.flowfile.repository.always.sync=false
+
+nifi.swap.manager.implementation=org.apache.nifi.controller.FileSystemSwapManager
+nifi.queue.swap.threshold=20000
+nifi.swap.in.period=5 sec
+nifi.swap.in.threads=1
+nifi.swap.out.period=5 sec
+nifi.swap.out.threads=4
+
+# Content Repository
+nifi.content.repository.implementation=org.apache.nifi.controller.repository.FileSystemRepository
+nifi.content.claim.max.appendable.size=10 MB
+nifi.content.claim.max.flow.files=100
+nifi.content.repository.directory.default=./content_repository
+nifi.content.repository.directory.content1=/tmp/foo/repo
+nifi.content.repository.archive.max.retention.period=12 hours
+nifi.content.repository.archive.max.usage.percentage=50%
+nifi.content.repository.archive.enabled=true
+nifi.content.repository.always.sync=false
+nifi.content.viewer.url=/nifi-content-viewer/
+
+# Provenance Repository Properties
+nifi.provenance.repository.implementation=org.apache.nifi.provenance.PersistentProvenanceRepository
+
+# Persistent Provenance Repository Properties
+nifi.provenance.repository.directory.default=./provenance_repository
+nifi.provenance.repository.max.storage.time=24 hours
+nifi.provenance.repository.max.storage.size=1 GB
+nifi.provenance.repository.rollover.time=30 secs
+nifi.provenance.repository.rollover.size=100 MB
+nifi.provenance.repository.query.threads=2
+nifi.provenance.repository.index.threads=1
+nifi.provenance.repository.compress.on.rollover=true
+nifi.provenance.repository.always.sync=false
+nifi.provenance.repository.journal.count=16
+# Comma-separated list of fields. Fields that are not indexed will not be searchable. Valid fields are:
+# EventType, FlowFileUUID, Filename, TransitURI, ProcessorID, AlternateIdentifierURI, Relationship, Details
+nifi.provenance.repository.indexed.fields=EventType, FlowFileUUID, Filename, ProcessorID, Relationship
+# FlowFile Attributes that should be indexed and made searchable.  Some examples to consider are filename, uuid, mime.type
+nifi.provenance.repository.indexed.attributes=
+# Large values for the shard size will result in more Java heap usage when searching the Provenance Repository
+# but should provide better performance
+nifi.provenance.repository.index.shard.size=500 MB
+# Indicates the maximum length that a FlowFile attribute can be when retrieving a Provenance Event from
+# the repository. If the length of any attribute exceeds this value, it will be truncated when the event is retrieved.
+nifi.provenance.repository.max.attribute.length=65536
+
+# Volatile Provenance Respository Properties
+nifi.provenance.repository.buffer.size=100000
+
+# Component Status Repository
+nifi.components.status.repository.implementation=org.apache.nifi.controller.status.history.VolatileComponentStatusRepository
+nifi.components.status.repository.buffer.size=1440
+nifi.components.status.snapshot.frequency=1 min
+
+# Site to Site properties
+nifi.remote.input.host=
+nifi.remote.input.secure=false
+nifi.remote.input.socket.port=
+nifi.remote.input.http.enabled=true
+nifi.remote.input.http.transaction.ttl=30 sec
+
+# web properties #
+nifi.web.war.directory=./lib
+nifi.web.http.host=
+nifi.web.http.port=8080
+nifi.web.https.host=
+nifi.web.https.port=
+nifi.web.jetty.working.directory=./work/jetty
+nifi.web.jetty.threads=200
+
+# security properties #
+nifi.sensitive.props.key=
+nifi.sensitive.props.key.protected=
+nifi.sensitive.props.algorithm=PBEWITHMD5AND256BITAES-CBC-OPENSSL
+nifi.sensitive.props.provider=BC
+nifi.sensitive.props.additional.keys=
+
+nifi.security.keystore=
+nifi.security.keystoreType=
+nifi.security.keystorePasswd=
+nifi.security.keyPasswd=
+nifi.security.truststore=
+nifi.security.truststoreType=
+nifi.security.truststorePasswd=
+nifi.security.needClientAuth=
+nifi.security.user.authorizer=file-provider
+nifi.security.user.login.identity.provider=
+nifi.security.ocsp.responder.url=
+nifi.security.ocsp.responder.certificate=
+
+# Identity Mapping Properties #
+# These properties allow normalizing user identities such that identities coming from different identity providers
+# (certificates, LDAP, Kerberos) can be treated the same internally in NiFi. The following example demonstrates normalizing
+# DNs from certificates and principals from Kerberos into a common identity string:
+#
+# nifi.security.identity.mapping.pattern.dn=^CN=(.*?), OU=(.*?), O=(.*?), L=(.*?), ST=(.*?), C=(.*?)$
+# nifi.security.identity.mapping.value.dn=$1@$2
+# nifi.security.identity.mapping.pattern.kerb=^(.*?)/instance@(.*?)$
+# nifi.security.identity.mapping.value.kerb=$1@$2
+
+# cluster common properties (all nodes must have same values) #
+nifi.cluster.protocol.heartbeat.interval=5 sec
+nifi.cluster.protocol.is.secure=false
+
+# cluster node properties (only configure for cluster nodes) #
+nifi.cluster.is.node=false
+nifi.cluster.node.address=
+nifi.cluster.node.protocol.port=
+nifi.cluster.node.protocol.threads=10
+nifi.cluster.node.event.history.size=25
+nifi.cluster.node.connection.timeout=5 sec
+nifi.cluster.node.read.timeout=5 sec
+nifi.cluster.firewall.file=
+nifi.cluster.flow.election.max.wait.time=5 mins
+nifi.cluster.flow.election.max.candidates=
+
+# zookeeper properties, used for cluster management #
+nifi.zookeeper.connect.string=
+nifi.zookeeper.connect.timeout=3 secs
+nifi.zookeeper.session.timeout=3 secs
+nifi.zookeeper.root.node=/nifi
+
+# kerberos #
+nifi.kerberos.krb5.file=
+
+# kerberos service principal #
+nifi.kerberos.service.principal=
+nifi.kerberos.service.keytab.location=
+
+# kerberos spnego principal #
+nifi.kerberos.spnego.principal=
+nifi.kerberos.spnego.keytab.location=
+nifi.kerberos.spnego.authentication.expiration=12 hours
+
+# external properties files for variable registry
+# supports a comma delimited list of file locations
+nifi.variable.registry.properties=
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi/blob/556f309d/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/pom.xml
----------------------------------------------------------------------
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 7f9b84b..192d4f1 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
@@ -45,6 +45,7 @@
                         <exclude>src/test/resources/nars/nar-with-versioning/META-INF/MANIFEST.MF</exclude>
                         <exclude>src/test/resources/nars/nar-without-versioning/META-INF/MANIFEST.MF</exclude>
                         <exclude>src/test/resources/nars/nar-without-dependency/META-INF/MANIFEST.MF</exclude>
+                        <exclude>src/test/resources/nars/nar-requires-cloning/META-INF/MANIFEST.MF</exclude>
                     </excludes>
                 </configuration>
             </plugin>

http://git-wip-us.apache.org/repos/asf/nifi/blob/556f309d/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/init/ConfigurableComponentInitializer.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/init/ConfigurableComponentInitializer.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/init/ConfigurableComponentInitializer.java
deleted file mode 100644
index 9a1149c..0000000
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/init/ConfigurableComponentInitializer.java
+++ /dev/null
@@ -1,45 +0,0 @@
-/*
- * 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.init;
-
-import org.apache.nifi.components.ConfigurableComponent;
-import org.apache.nifi.reporting.InitializationException;
-
-/**
- * An interface for initializing and tearing down a ConfigurableComponent. It is up to the
- * implementer to call "init" so that you can call
- * ConfigurableComponent.getPropertyDescriptors()
- *
- */
-public interface ConfigurableComponentInitializer {
-
-    /**
-     * Initializes a configurable component to the point that you can call
-     * getPropertyDescriptors() on it
-     *
-     * @param component the component to initialize
-     * @throws InitializationException if the component could not be initialized
-     */
-    void initialize(ConfigurableComponent component) throws InitializationException;
-
-    /**
-     * Calls the lifecycle methods that should be called when a flow is shutdown.
-     *
-     * @param component the component to initialize
-     */
-    void teardown(ConfigurableComponent component);
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/556f309d/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/init/ConfigurableComponentInitializerFactory.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/init/ConfigurableComponentInitializerFactory.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/init/ConfigurableComponentInitializerFactory.java
deleted file mode 100644
index f6ab922..0000000
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/init/ConfigurableComponentInitializerFactory.java
+++ /dev/null
@@ -1,44 +0,0 @@
-/*
- * 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.init;
-
-import org.apache.nifi.components.ConfigurableComponent;
-import org.apache.nifi.controller.ControllerService;
-import org.apache.nifi.processor.Processor;
-import org.apache.nifi.reporting.ReportingTask;
-
-public class ConfigurableComponentInitializerFactory {
-
-    /**
-     * Returns a ConfigurableComponentInitializer for the type of component.
-     * Currently Processor, ControllerService and ReportingTask are supported.
-     *
-     * @param componentClass the class that requires a ConfigurableComponentInitializer
-     * @return a ConfigurableComponentInitializer capable of initializing that specific type of class
-     */
-    public static ConfigurableComponentInitializer createComponentInitializer(final Class<? extends ConfigurableComponent> componentClass) {
-        if (Processor.class.isAssignableFrom(componentClass)) {
-            return new ProcessorInitializer();
-        } else if (ControllerService.class.isAssignableFrom(componentClass)) {
-            return new ControllerServiceInitializer();
-        } else if (ReportingTask.class.isAssignableFrom(componentClass)) {
-            return new ReportingTaskingInitializer();
-        }
-
-        return null;
-    }
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/556f309d/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/init/ControllerServiceInitializer.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/init/ControllerServiceInitializer.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/init/ControllerServiceInitializer.java
deleted file mode 100644
index 5939b96..0000000
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/init/ControllerServiceInitializer.java
+++ /dev/null
@@ -1,59 +0,0 @@
-/*
- * 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.init;
-
-import org.apache.nifi.annotation.lifecycle.OnShutdown;
-import org.apache.nifi.components.ConfigurableComponent;
-import org.apache.nifi.controller.ControllerService;
-import org.apache.nifi.controller.ControllerServiceInitializationContext;
-import org.apache.nifi.logging.ComponentLog;
-import org.apache.nifi.mock.MockComponentLogger;
-import org.apache.nifi.mock.MockConfigurationContext;
-import org.apache.nifi.mock.MockControllerServiceInitializationContext;
-import org.apache.nifi.nar.ExtensionManager;
-import org.apache.nifi.nar.NarCloseable;
-import org.apache.nifi.reporting.InitializationException;
-
-/**
- * Initializes a ControllerService using a MockControllerServiceInitializationContext
- *
- *
- */
-public class ControllerServiceInitializer implements ConfigurableComponentInitializer {
-
-    @Override
-    public void initialize(ConfigurableComponent component) throws InitializationException {
-        ControllerService controllerService = (ControllerService) component;
-        ControllerServiceInitializationContext context = new MockControllerServiceInitializationContext();
-        try (NarCloseable narCloseable = NarCloseable.withComponentNarLoader(component.getClass(), context.getIdentifier())) {
-            controllerService.initialize(context);
-        }
-    }
-
-    @Override
-    public void teardown(ConfigurableComponent component) {
-        try (NarCloseable narCloseable = NarCloseable.withComponentNarLoader(component.getClass(), component.getIdentifier())) {
-            ControllerService controllerService = (ControllerService) component;
-
-            final ComponentLog logger = new MockComponentLogger();
-            final MockConfigurationContext context = new MockConfigurationContext();
-            ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnShutdown.class, controllerService, logger, context);
-        } finally {
-            ExtensionManager.removeInstanceClassLoaderIfExists(component.getIdentifier());
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/556f309d/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/init/ProcessorInitializer.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/init/ProcessorInitializer.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/init/ProcessorInitializer.java
deleted file mode 100644
index 3274f6e..0000000
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/init/ProcessorInitializer.java
+++ /dev/null
@@ -1,58 +0,0 @@
-/*
- * 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.init;
-
-import org.apache.nifi.annotation.lifecycle.OnShutdown;
-import org.apache.nifi.components.ConfigurableComponent;
-import org.apache.nifi.logging.ComponentLog;
-import org.apache.nifi.mock.MockComponentLogger;
-import org.apache.nifi.mock.MockProcessContext;
-import org.apache.nifi.mock.MockProcessorInitializationContext;
-import org.apache.nifi.nar.ExtensionManager;
-import org.apache.nifi.nar.NarCloseable;
-import org.apache.nifi.processor.Processor;
-import org.apache.nifi.processor.ProcessorInitializationContext;
-
-/**
- * Initializes a Processor using a MockProcessorInitializationContext
- *
- *
- */
-public class ProcessorInitializer implements ConfigurableComponentInitializer {
-
-    @Override
-    public void initialize(ConfigurableComponent component) {
-        Processor processor = (Processor) component;
-        ProcessorInitializationContext initializationContext = new MockProcessorInitializationContext();
-        try (NarCloseable narCloseable = NarCloseable.withComponentNarLoader(component.getClass(), initializationContext.getIdentifier())) {
-            processor.initialize(initializationContext);
-        }
-    }
-
-    @Override
-    public void teardown(ConfigurableComponent component) {
-        Processor processor = (Processor) component;
-        try (NarCloseable narCloseable = NarCloseable.withComponentNarLoader(component.getClass(), component.getIdentifier())) {
-
-            final ComponentLog logger = new MockComponentLogger();
-            final MockProcessContext context = new MockProcessContext();
-            ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnShutdown.class, processor, logger, context);
-        } finally {
-            ExtensionManager.removeInstanceClassLoaderIfExists(component.getIdentifier());
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/556f309d/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/init/ReflectionUtils.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/init/ReflectionUtils.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/init/ReflectionUtils.java
deleted file mode 100644
index 22420bd..0000000
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/init/ReflectionUtils.java
+++ /dev/null
@@ -1,133 +0,0 @@
-/*
- * 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.init;
-
-import org.apache.nifi.logging.ComponentLog;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.lang.annotation.Annotation;
-import java.lang.reflect.InvocationTargetException;
-import java.lang.reflect.Method;
-
-/**
- * This class is a copy of org.apache.nifi.util.ReflectionUtils. Ultimately the
- * documentation generation component should be moved to a place where it can
- * depend on this directly instead of copying it in.
- *
- *
- */
-public class ReflectionUtils {
-
-    private final static Logger LOG = LoggerFactory.getLogger(ReflectionUtils.class);
-
-    /**
-     * Invokes all methods on the given instance that have been annotated with
-     * the given annotation. If the signature of the method that is defined in
-     * <code>instance</code> uses 1 or more parameters, those parameters must be
-     * specified by the <code>args</code> parameter. However, if more arguments
-     * are supplied by the <code>args</code> parameter than needed, the extra
-     * arguments will be ignored.
-     *
-     * @param annotation annotation
-     * @param instance instance
-     * @param logger the ComponentLog to use for logging any errors. If null,
-     * will use own logger, but that will not generate bulletins or easily tie
-     * to the Processor's log messages.
-     * @param args args
-     * @return <code>true</code> if all appropriate methods were invoked and
-     * returned without throwing an Exception, <code>false</code> if one of the
-     * methods threw an Exception or could not be invoked; if <code>false</code>
-     * is returned, an error will have been logged.
-     */
-    public static boolean quietlyInvokeMethodsWithAnnotation(
-            final Class<? extends Annotation> annotation, final Object instance, final ComponentLog logger, final Object... args) {
-
-        for (final Method method : instance.getClass().getMethods()) {
-            if (method.isAnnotationPresent(annotation)) {
-
-                final boolean isAccessible = method.isAccessible();
-                method.setAccessible(true);
-
-                try {
-                    final Class<?>[] argumentTypes = method.getParameterTypes();
-                    if (argumentTypes.length > args.length) {
-                        if (logger == null) {
-                            LOG.error("Unable to invoke method {} on {} because method expects {} parameters but only {} were given",
-                                    new Object[]{method.getName(), instance, argumentTypes.length, args.length});
-                        } else {
-                            logger.error("Unable to invoke method {} on {} because method expects {} parameters but only {} were given",
-                                    new Object[]{method.getName(), instance, argumentTypes.length, args.length});
-                        }
-
-                        return false;
-                    }
-
-                    for (int i = 0; i < argumentTypes.length; i++) {
-                        final Class<?> argType = argumentTypes[i];
-                        if (!argType.isAssignableFrom(args[i].getClass())) {
-                            if (logger == null) {
-                                LOG.error("Unable to invoke method {} on {} because method parameter {} is expected to be of type {} but argument passed was of type {}",
-                                        new Object[]{method.getName(), instance, i, argType, args[i].getClass()});
-                            } else {
-                                logger.error("Unable to invoke method {} on {} because method parameter {} is expected to be of type {} but argument passed was of type {}",
-                                        new Object[]{method.getName(), instance, i, argType, args[i].getClass()});
-                            }
-
-                            return false;
-                        }
-                    }
-
-                    try {
-                        if (argumentTypes.length == args.length) {
-                            method.invoke(instance, args);
-                        } else {
-                            final Object[] argsToPass = new Object[argumentTypes.length];
-                            for (int i = 0; i < argsToPass.length; i++) {
-                                argsToPass[i] = args[i];
-                            }
-
-                            method.invoke(instance, argsToPass);
-                        }
-                    } catch (final InvocationTargetException ite) {
-                        if (logger == null) {
-                            LOG.error("Unable to invoke method {} on {} due to {}", new Object[]{method.getName(), instance, ite.getCause()});
-                            LOG.error("", ite.getCause());
-                        } else {
-                            logger.error("Unable to invoke method {} on {} due to {}", new Object[]{method.getName(), instance, ite.getCause()});
-                        }
-                    } catch (final IllegalAccessException | IllegalArgumentException t) {
-                        if (logger == null) {
-                            LOG.error("Unable to invoke method {} on {} due to {}", new Object[]{method.getName(), instance, t});
-                            LOG.error("", t);
-                        } else {
-                            logger.error("Unable to invoke method {} on {} due to {}", new Object[]{method.getName(), instance, t});
-                        }
-
-                        return false;
-                    }
-                } finally {
-                    if (!isAccessible) {
-                        method.setAccessible(false);
-                    }
-                }
-            }
-        }
-
-        return true;
-    }
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/556f309d/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/init/ReportingTaskingInitializer.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/init/ReportingTaskingInitializer.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/init/ReportingTaskingInitializer.java
deleted file mode 100644
index 546e67c..0000000
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/init/ReportingTaskingInitializer.java
+++ /dev/null
@@ -1,57 +0,0 @@
-/*
- * 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.init;
-
-import org.apache.nifi.annotation.lifecycle.OnShutdown;
-import org.apache.nifi.components.ConfigurableComponent;
-import org.apache.nifi.mock.MockComponentLogger;
-import org.apache.nifi.mock.MockConfigurationContext;
-import org.apache.nifi.mock.MockReportingInitializationContext;
-import org.apache.nifi.nar.ExtensionManager;
-import org.apache.nifi.nar.NarCloseable;
-import org.apache.nifi.reporting.InitializationException;
-import org.apache.nifi.reporting.ReportingInitializationContext;
-import org.apache.nifi.reporting.ReportingTask;
-
-/**
- * Initializes a ReportingTask using a MockReportingInitializationContext;
- *
- *
- */
-public class ReportingTaskingInitializer implements ConfigurableComponentInitializer {
-
-    @Override
-    public void initialize(ConfigurableComponent component) throws InitializationException {
-        ReportingTask reportingTask = (ReportingTask) component;
-        ReportingInitializationContext context = new MockReportingInitializationContext();
-        try (NarCloseable narCloseable = NarCloseable.withComponentNarLoader(component.getClass(), context.getIdentifier())) {
-            reportingTask.initialize(context);
-        }
-    }
-
-    @Override
-    public void teardown(ConfigurableComponent component) {
-        ReportingTask reportingTask = (ReportingTask) component;
-        try (NarCloseable narCloseable = NarCloseable.withComponentNarLoader(component.getClass(), component.getIdentifier())) {
-
-            final MockConfigurationContext context = new MockConfigurationContext();
-            ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnShutdown.class, reportingTask, new MockComponentLogger(), context);
-        } finally {
-            ExtensionManager.removeInstanceClassLoaderIfExists(component.getIdentifier());
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/556f309d/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/mock/MockComponentLogger.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/mock/MockComponentLogger.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/mock/MockComponentLogger.java
deleted file mode 100644
index 920d7eb..0000000
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/mock/MockComponentLogger.java
+++ /dev/null
@@ -1,258 +0,0 @@
-/*
- * 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.mock;
-
-import org.apache.nifi.logging.ComponentLog;
-import org.apache.nifi.logging.LogLevel;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Stubs out the functionality of a ComponentLog so that it can
- * be used during initialization of a component.
- *
- */
-public class MockComponentLogger implements ComponentLog {
-
-    private static final Logger logger = LoggerFactory
-            .getLogger(MockComponentLogger.class);
-
-    @Override
-    public void warn(String msg, Throwable t) {
-        logger.warn(msg, t);
-    }
-
-    @Override
-    public void warn(String msg, Object[] os) {
-        logger.warn(msg, os);
-    }
-
-    @Override
-    public void warn(String msg, Object[] os, Throwable t) {
-        logger.warn(msg, os);
-        logger.warn("", t);
-    }
-
-    @Override
-    public void warn(String msg) {
-        logger.warn(msg);
-    }
-
-    @Override
-    public void trace(String msg, Throwable t) {
-        logger.trace(msg, t);
-    }
-
-    @Override
-    public void trace(String msg, Object[] os) {
-        logger.trace(msg, os);
-    }
-
-    @Override
-    public void trace(String msg) {
-        logger.trace(msg);
-    }
-
-    @Override
-    public void trace(String msg, Object[] os, Throwable t) {
-        logger.trace(msg, os);
-        logger.trace("", t);
-    }
-
-    @Override
-    public boolean isWarnEnabled() {
-        return logger.isWarnEnabled();
-    }
-
-    @Override
-    public boolean isTraceEnabled() {
-        return logger.isTraceEnabled();
-    }
-
-    @Override
-    public boolean isInfoEnabled() {
-        return logger.isInfoEnabled();
-    }
-
-    @Override
-    public boolean isErrorEnabled() {
-        return logger.isErrorEnabled();
-    }
-
-    @Override
-    public boolean isDebugEnabled() {
-        return logger.isDebugEnabled();
-    }
-
-    @Override
-    public void info(String msg, Throwable t) {
-        logger.info(msg, t);
-    }
-
-    @Override
-    public void info(String msg, Object[] os) {
-        logger.info(msg, os);
-    }
-
-    @Override
-    public void info(String msg) {
-        logger.info(msg);
-
-    }
-
-    @Override
-    public void info(String msg, Object[] os, Throwable t) {
-        logger.trace(msg, os);
-        logger.trace("", t);
-
-    }
-
-    @Override
-    public String getName() {
-        return logger.getName();
-    }
-
-    @Override
-    public void error(String msg, Throwable t) {
-        logger.error(msg, t);
-    }
-
-    @Override
-    public void error(String msg, Object[] os) {
-        logger.error(msg, os);
-    }
-
-    @Override
-    public void error(String msg) {
-        logger.error(msg);
-    }
-
-    @Override
-    public void error(String msg, Object[] os, Throwable t) {
-        logger.error(msg, os);
-        logger.error("", t);
-    }
-
-    @Override
-    public void debug(String msg, Throwable t) {
-        logger.debug(msg, t);
-    }
-
-    @Override
-    public void debug(String msg, Object[] os) {
-        logger.debug(msg, os);
-    }
-
-    @Override
-    public void debug(String msg, Object[] os, Throwable t) {
-        logger.debug(msg, os);
-        logger.debug("", t);
-    }
-
-    @Override
-    public void debug(String msg) {
-        logger.debug(msg);
-    }
-
-    @Override
-    public void log(LogLevel level, String msg, Throwable t) {
-        switch (level) {
-            case DEBUG:
-                debug(msg, t);
-                break;
-            case ERROR:
-            case FATAL:
-                error(msg, t);
-                break;
-            case INFO:
-                info(msg, t);
-                break;
-            case TRACE:
-                trace(msg, t);
-                break;
-            case WARN:
-                warn(msg, t);
-                break;
-        }
-    }
-
-    @Override
-    public void log(LogLevel level, String msg, Object[] os) {
-        switch (level) {
-            case DEBUG:
-                debug(msg, os);
-                break;
-            case ERROR:
-            case FATAL:
-                error(msg, os);
-                break;
-            case INFO:
-                info(msg, os);
-                break;
-            case TRACE:
-                trace(msg, os);
-                break;
-            case WARN:
-                warn(msg, os);
-                break;
-        }
-    }
-
-    @Override
-    public void log(LogLevel level, String msg) {
-        switch (level) {
-            case DEBUG:
-                debug(msg);
-                break;
-            case ERROR:
-            case FATAL:
-                error(msg);
-                break;
-            case INFO:
-                info(msg);
-                break;
-            case TRACE:
-                trace(msg);
-                break;
-            case WARN:
-                warn(msg);
-                break;
-        }
-    }
-
-    @Override
-    public void log(LogLevel level, String msg, Object[] os, Throwable t) {
-        switch (level) {
-            case DEBUG:
-                debug(msg, os, t);
-                break;
-            case ERROR:
-            case FATAL:
-                error(msg, os, t);
-                break;
-            case INFO:
-                info(msg, os, t);
-                break;
-            case TRACE:
-                trace(msg, os, t);
-                break;
-            case WARN:
-                warn(msg, os, t);
-                break;
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/556f309d/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/mock/MockConfigurationContext.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/mock/MockConfigurationContext.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/mock/MockConfigurationContext.java
deleted file mode 100644
index d1e73fb..0000000
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/mock/MockConfigurationContext.java
+++ /dev/null
@@ -1,48 +0,0 @@
-/*
- * 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.mock;
-
-import org.apache.nifi.components.PropertyDescriptor;
-import org.apache.nifi.components.PropertyValue;
-import org.apache.nifi.controller.ConfigurationContext;
-
-import java.util.Collections;
-import java.util.Map;
-import java.util.concurrent.TimeUnit;
-
-public class MockConfigurationContext implements ConfigurationContext {
-
-    @Override
-    public PropertyValue getProperty(PropertyDescriptor property) {
-        return null;
-    }
-
-    @Override
-    public Map<PropertyDescriptor, String> getProperties() {
-        return Collections.emptyMap();
-    }
-
-    @Override
-    public String getSchedulingPeriod() {
-        return "0 secs";
-    }
-
-    @Override
-    public Long getSchedulingPeriod(final TimeUnit timeUnit) {
-        return 0L;
-    }
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/556f309d/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/mock/MockControllerServiceInitializationContext.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/mock/MockControllerServiceInitializationContext.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/mock/MockControllerServiceInitializationContext.java
deleted file mode 100644
index b111ad2..0000000
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/mock/MockControllerServiceInitializationContext.java
+++ /dev/null
@@ -1,68 +0,0 @@
-/*
- * 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.mock;
-
-import org.apache.nifi.components.state.StateManager;
-import org.apache.nifi.controller.ControllerServiceInitializationContext;
-import org.apache.nifi.controller.ControllerServiceLookup;
-import org.apache.nifi.logging.ComponentLog;
-
-import java.io.File;
-
-/**
- * A Mock ControllerServiceInitializationContext so that ControllerServices can
- * be initialized for the purpose of generating documentation.
- *
- *
- */
-public class MockControllerServiceInitializationContext implements ControllerServiceInitializationContext {
-
-    @Override
-    public String getIdentifier() {
-        return "mock-controller-service";
-    }
-
-    @Override
-    public ControllerServiceLookup getControllerServiceLookup() {
-        return new MockControllerServiceLookup();
-    }
-
-    @Override
-    public ComponentLog getLogger() {
-        return new MockComponentLogger();
-    }
-
-    @Override
-    public StateManager getStateManager() {
-        return null;
-    }
-
-    @Override
-    public String getKerberosServicePrincipal() {
-        return null;
-    }
-
-    @Override
-    public File getKerberosServiceKeytab() {
-        return null;
-    }
-
-    @Override
-    public File getKerberosConfigurationFile() {
-        return null;
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi/blob/556f309d/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/mock/MockControllerServiceLookup.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/mock/MockControllerServiceLookup.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/mock/MockControllerServiceLookup.java
deleted file mode 100644
index 5307ac4..0000000
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/mock/MockControllerServiceLookup.java
+++ /dev/null
@@ -1,63 +0,0 @@
-/*
- * 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.mock;
-
-import org.apache.nifi.controller.ControllerService;
-import org.apache.nifi.controller.ControllerServiceLookup;
-
-import java.util.Collections;
-import java.util.Set;
-
-/**
- * A Mock ControllerServiceLookup that can be used so that
- * ConfigurableComponents can be initialized for the purpose of generating
- * documentation
- *
- *
- */
-public class MockControllerServiceLookup implements ControllerServiceLookup {
-
-    @Override
-    public ControllerService getControllerService(final String serviceIdentifier) {
-        return null;
-    }
-
-    @Override
-    public boolean isControllerServiceEnabled(final String serviceIdentifier) {
-        return false;
-    }
-
-    @Override
-    public boolean isControllerServiceEnabled(final ControllerService service) {
-        return false;
-    }
-
-    @Override
-    public Set<String> getControllerServiceIdentifiers(final Class<? extends ControllerService> serviceType) throws IllegalArgumentException {
-        return Collections.emptySet();
-    }
-
-    @Override
-    public boolean isControllerServiceEnabling(final String serviceIdentifier) {
-        return false;
-    }
-
-    @Override
-    public String getControllerServiceName(final String serviceIdentifier) {
-        return serviceIdentifier;
-    }
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/556f309d/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/mock/MockNodeTypeProvider.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/mock/MockNodeTypeProvider.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/mock/MockNodeTypeProvider.java
deleted file mode 100644
index 61390e1..0000000
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/mock/MockNodeTypeProvider.java
+++ /dev/null
@@ -1,40 +0,0 @@
-/*
- * 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.mock;
-
-import org.apache.nifi.controller.NodeTypeProvider;
-
-/**
- * A Mock NodeTypeProvider that can be used so that
- * ConfigurableComponents can be initialized for the purpose of generating
- * documentation
- *
- *
- */
-public class MockNodeTypeProvider implements NodeTypeProvider {
-
-    @Override
-    public boolean isClustered() {
-        return false;
-    }
-
-    @Override
-    public boolean isPrimary() {
-        return false;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/556f309d/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/mock/MockProcessContext.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/mock/MockProcessContext.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/mock/MockProcessContext.java
deleted file mode 100644
index cf2e2cf..0000000
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/mock/MockProcessContext.java
+++ /dev/null
@@ -1,116 +0,0 @@
-/*
- * 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.mock;
-
-import org.apache.nifi.components.PropertyDescriptor;
-import org.apache.nifi.components.PropertyValue;
-import org.apache.nifi.components.state.StateManager;
-import org.apache.nifi.controller.ControllerServiceLookup;
-import org.apache.nifi.processor.ProcessContext;
-import org.apache.nifi.processor.Relationship;
-
-import java.util.Collections;
-import java.util.Map;
-import java.util.Set;
-
-public class MockProcessContext implements ProcessContext {
-
-    @Override
-    public PropertyValue getProperty(PropertyDescriptor descriptor) {
-        return null;
-    }
-
-    @Override
-    public PropertyValue getProperty(String propertyName) {
-        return null;
-    }
-
-    @Override
-    public PropertyValue newPropertyValue(String rawValue) {
-        return null;
-    }
-
-    @Override
-    public void yield() {
-
-    }
-
-    @Override
-    public int getMaxConcurrentTasks() {
-        return 0;
-    }
-
-    @Override
-    public String getAnnotationData() {
-        return "";
-    }
-
-    @Override
-    public Map<PropertyDescriptor, String> getProperties() {
-        return Collections.emptyMap();
-    }
-
-    @Override
-    public String encrypt(String unencrypted) {
-        return unencrypted;
-    }
-
-    @Override
-    public String decrypt(String encrypted) {
-        return encrypted;
-    }
-
-    @Override
-    public ControllerServiceLookup getControllerServiceLookup() {
-        return new MockControllerServiceLookup();
-    }
-
-    @Override
-    public Set<Relationship> getAvailableRelationships() {
-        return Collections.emptySet();
-    }
-
-    @Override
-    public boolean hasIncomingConnection() {
-        return true;
-    }
-
-    @Override
-    public boolean hasNonLoopConnection() {
-        return true;
-    }
-
-    @Override
-    public boolean hasConnection(Relationship relationship) {
-        return false;
-    }
-
-    @Override
-    public boolean isExpressionLanguagePresent(PropertyDescriptor property) {
-        return false;
-    }
-
-    @Override
-    public StateManager getStateManager() {
-        return null;
-    }
-
-    @Override
-    public String getName() {
-        return null;
-    }
-}
\ No newline at end of file


[3/7] nifi git commit: NIFI-3520 Refactoring instance class loading - Fixing FlowController to use appropriate class loader when instantiating processor - Updating ExtensionManager to leverage new flag in MANIFEST from NAR plugin - Adding ReloadComponent

Posted by mc...@apache.org.
http://git-wip-us.apache.org/repos/asf/nifi/blob/556f309d/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/mock/MockProcessorInitializationContext.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/mock/MockProcessorInitializationContext.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/mock/MockProcessorInitializationContext.java
deleted file mode 100644
index d9320b2..0000000
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/mock/MockProcessorInitializationContext.java
+++ /dev/null
@@ -1,68 +0,0 @@
-/*
- * 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.mock;
-
-import org.apache.nifi.controller.ControllerServiceLookup;
-import org.apache.nifi.controller.NodeTypeProvider;
-import org.apache.nifi.logging.ComponentLog;
-import org.apache.nifi.processor.ProcessorInitializationContext;
-
-import java.io.File;
-
-/**
- * A Mock ProcessorInitializationContext that can be used so that Processors can
- * be initialized for the purpose of generating documentation.
- *
- *
- */
-public class MockProcessorInitializationContext implements ProcessorInitializationContext {
-
-    @Override
-    public String getIdentifier() {
-        return "mock-processor";
-    }
-
-    @Override
-    public ComponentLog getLogger() {
-        return new MockComponentLogger();
-    }
-
-    @Override
-    public ControllerServiceLookup getControllerServiceLookup() {
-        return new MockControllerServiceLookup();
-    }
-
-    @Override
-    public NodeTypeProvider getNodeTypeProvider() {
-        return new MockNodeTypeProvider();
-    }
-
-    @Override
-    public String getKerberosServicePrincipal() {
-        return null;
-    }
-
-    @Override
-    public File getKerberosServiceKeytab() {
-        return null;
-    }
-
-    @Override
-    public File getKerberosConfigurationFile() {
-        return null;
-    }
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/556f309d/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/mock/MockReportingInitializationContext.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/mock/MockReportingInitializationContext.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/mock/MockReportingInitializationContext.java
deleted file mode 100644
index 630c657..0000000
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/mock/MockReportingInitializationContext.java
+++ /dev/null
@@ -1,83 +0,0 @@
-/*
- * 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.mock;
-
-import org.apache.nifi.controller.ControllerServiceLookup;
-import org.apache.nifi.logging.ComponentLog;
-import org.apache.nifi.reporting.ReportingInitializationContext;
-import org.apache.nifi.scheduling.SchedulingStrategy;
-
-import java.io.File;
-import java.util.concurrent.TimeUnit;
-
-/**
- * A Mock ReportingInitializationContext that can be used to initialize a
- * ReportingTask for the purposes of documentation generation.
- *
- */
-public class MockReportingInitializationContext implements ReportingInitializationContext {
-
-    @Override
-    public String getIdentifier() {
-        return "mock-reporting-task";
-    }
-
-    @Override
-    public String getName() {
-        return "";
-    }
-
-    @Override
-    public long getSchedulingPeriod(TimeUnit timeUnit) {
-        return 0;
-    }
-
-    @Override
-    public ControllerServiceLookup getControllerServiceLookup() {
-        return new MockControllerServiceLookup();
-    }
-
-    @Override
-    public String getSchedulingPeriod() {
-        return "";
-    }
-
-    @Override
-    public SchedulingStrategy getSchedulingStrategy() {
-        return SchedulingStrategy.TIMER_DRIVEN;
-    }
-
-    @Override
-    public ComponentLog getLogger() {
-        return new MockComponentLogger();
-    }
-
-    @Override
-    public String getKerberosServicePrincipal() {
-        return null;
-    }
-
-    @Override
-    public File getKerberosServiceKeytab() {
-        return null;
-    }
-
-    @Override
-    public File getKerberosConfigurationFile() {
-        return null;
-    }
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/556f309d/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/nar/ExtensionManager.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/nar/ExtensionManager.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/nar/ExtensionManager.java
deleted file mode 100644
index 1cff3af..0000000
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/nar/ExtensionManager.java
+++ /dev/null
@@ -1,478 +0,0 @@
-/*
- * 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 org.apache.nifi.annotation.behavior.RequiresInstanceClassLoading;
-import org.apache.nifi.authentication.LoginIdentityProvider;
-import org.apache.nifi.authorization.Authorizer;
-import org.apache.nifi.bundle.Bundle;
-import org.apache.nifi.bundle.BundleCoordinate;
-import org.apache.nifi.bundle.BundleDetails;
-import org.apache.nifi.components.ConfigurableComponent;
-import org.apache.nifi.components.PropertyDescriptor;
-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;
-import org.apache.nifi.controller.repository.FlowFileSwapManager;
-import org.apache.nifi.controller.status.history.ComponentStatusRepository;
-import org.apache.nifi.flowfile.FlowFilePrioritizer;
-import org.apache.nifi.init.ConfigurableComponentInitializer;
-import org.apache.nifi.init.ConfigurableComponentInitializerFactory;
-import org.apache.nifi.processor.Processor;
-import org.apache.nifi.provenance.ProvenanceRepository;
-import org.apache.nifi.reporting.InitializationException;
-import org.apache.nifi.reporting.ReportingTask;
-import org.apache.nifi.util.NiFiProperties;
-import org.apache.nifi.util.StringUtils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.File;
-import java.io.IOException;
-import java.net.URL;
-import java.net.URLClassLoader;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.ServiceLoader;
-import java.util.Set;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.stream.Collectors;
-
-/**
- * Scans through the classpath to load all FlowFileProcessors, FlowFileComparators, and ReportingTasks using the service provider API and running through all classloaders (root, NARs).
- *
- * @ThreadSafe - is immutable
- */
-@SuppressWarnings("rawtypes")
-public class ExtensionManager {
-
-    private static final Logger logger = LoggerFactory.getLogger(ExtensionManager.class);
-
-    public static final BundleCoordinate SYSTEM_BUNDLE_COORDINATE = new BundleCoordinate(
-            BundleCoordinate.DEFAULT_GROUP, "system", BundleCoordinate.DEFAULT_VERSION);
-
-    // Maps a service definition (interface) to those classes that implement the interface
-    private static final Map<Class, Set<Class>> definitionMap = new HashMap<>();
-
-    private static final Map<String, List<Bundle>> classNameBundleLookup = new HashMap<>();
-    private static final Map<BundleCoordinate, Bundle> bundleCoordinateBundleLookup = new HashMap<>();
-    private static final Map<ClassLoader, Bundle> classLoaderBundleLookup = new HashMap<>();
-
-    private static final Set<String> requiresInstanceClassLoading = new HashSet<>();
-    private static final Map<String, ClassLoader> instanceClassloaderLookup = new ConcurrentHashMap<>();
-
-    static {
-        definitionMap.put(Processor.class, new HashSet<>());
-        definitionMap.put(FlowFilePrioritizer.class, new HashSet<>());
-        definitionMap.put(ReportingTask.class, new HashSet<>());
-        definitionMap.put(ControllerService.class, new HashSet<>());
-        definitionMap.put(Authorizer.class, new HashSet<>());
-        definitionMap.put(LoginIdentityProvider.class, new HashSet<>());
-        definitionMap.put(ProvenanceRepository.class, new HashSet<>());
-        definitionMap.put(ComponentStatusRepository.class, new HashSet<>());
-        definitionMap.put(FlowFileRepository.class, new HashSet<>());
-        definitionMap.put(FlowFileSwapManager.class, new HashSet<>());
-        definitionMap.put(ContentRepository.class, new HashSet<>());
-        definitionMap.put(StateProvider.class, new HashSet<>());
-    }
-
-    /**
-     * Loads all FlowFileProcessor, FlowFileComparator, ReportingTask class types that can be found on the bootstrap classloader and by creating classloaders for all NARs found within the classpath.
-     * @param narBundles the bundles to scan through in search of extensions
-     */
-    public static void discoverExtensions(final Bundle systemBundle, final Set<Bundle> narBundles) {
-        // get the current context class loader
-        ClassLoader currentContextClassLoader = Thread.currentThread().getContextClassLoader();
-
-        // load the system bundle first so that any extensions found in JARs directly in lib will be registered as
-        // being from the system bundle and not from all the other NARs
-        loadExtensions(systemBundle);
-        bundleCoordinateBundleLookup.put(systemBundle.getBundleDetails().getCoordinate(), systemBundle);
-
-        // consider each nar class loader
-        for (final Bundle bundle : narBundles) {
-            // Must set the context class loader to the nar classloader itself
-            // so that static initialization techniques that depend on the context class loader will work properly
-            final ClassLoader ncl = bundle.getClassLoader();
-            Thread.currentThread().setContextClassLoader(ncl);
-            loadExtensions(bundle);
-
-            // Create a look-up from coordinate to bundle
-            bundleCoordinateBundleLookup.put(bundle.getBundleDetails().getCoordinate(), bundle);
-        }
-
-        // restore the current context class loader if appropriate
-        if (currentContextClassLoader != null) {
-            Thread.currentThread().setContextClassLoader(currentContextClassLoader);
-        }
-    }
-
-    /**
-     * Returns a bundle representing the system class loader.
-     *
-     * @param niFiProperties a NiFiProperties instance which will be used to obtain the default NAR library path,
-     *                       which will become the working directory of the returned bundle
-     * @return a bundle for the system class loader
-     */
-    public static Bundle createSystemBundle(final NiFiProperties niFiProperties) {
-        final ClassLoader systemClassLoader = ClassLoader.getSystemClassLoader();
-
-        final String narLibraryDirectory = niFiProperties.getProperty(NiFiProperties.NAR_LIBRARY_DIRECTORY);
-        if (StringUtils.isBlank(narLibraryDirectory)) {
-            throw new IllegalStateException("Unable to create system bundle because " + NiFiProperties.NAR_LIBRARY_DIRECTORY + " was null or empty");
-        }
-
-        final BundleDetails systemBundleDetails = new BundleDetails.Builder()
-                .workingDir(new File(narLibraryDirectory))
-                .coordinate(SYSTEM_BUNDLE_COORDINATE)
-                .build();
-
-        return new Bundle(systemBundleDetails, systemClassLoader);
-    }
-
-    /**
-     * Loads extensions from the specified bundle.
-     *
-     * @param bundle from which to load extensions
-     */
-    @SuppressWarnings("unchecked")
-    private static void loadExtensions(final Bundle bundle) {
-        for (final Map.Entry<Class, Set<Class>> entry : definitionMap.entrySet()) {
-            final boolean isControllerService = ControllerService.class.equals(entry.getKey());
-            final boolean isProcessor = Processor.class.equals(entry.getKey());
-            final boolean isReportingTask = ReportingTask.class.equals(entry.getKey());
-
-            final ServiceLoader<?> serviceLoader = ServiceLoader.load(entry.getKey(), bundle.getClassLoader());
-            for (final Object o : serviceLoader) {
-                // only consider extensions discovered directly in this bundle
-                boolean registerExtension = bundle.getClassLoader().equals(o.getClass().getClassLoader());
-
-                if (registerExtension) {
-                    final Class extensionType = o.getClass();
-                    if (isControllerService && !checkControllerServiceEligibility(extensionType)) {
-                        registerExtension = false;
-                        logger.error(String.format(
-                                "Skipping Controller Service %s because it is bundled with its supporting APIs and requires instance class loading.", extensionType.getName()));
-                    }
-
-                    final boolean canReferenceControllerService = (isControllerService || isProcessor || isReportingTask) && o instanceof ConfigurableComponent;
-                    if (canReferenceControllerService && !checkControllerServiceReferenceEligibility((ConfigurableComponent) o, bundle.getClassLoader())) {
-                        registerExtension = false;
-                        logger.error(String.format(
-                                "Skipping component %s because it is bundled with its referenced Controller Service APIs and requires instance class loading.", extensionType.getName()));
-                    }
-
-                    if (registerExtension) {
-                        registerServiceClass(o.getClass(), classNameBundleLookup, bundle, entry.getValue());
-                    }
-                }
-            }
-
-            classLoaderBundleLookup.put(bundle.getClassLoader(), bundle);
-        }
-    }
-
-    private static boolean checkControllerServiceReferenceEligibility(final ConfigurableComponent component, final ClassLoader classLoader) {
-        // if the extension does not require instance classloading, its eligible
-        final boolean requiresInstanceClassLoading = component.getClass().isAnnotationPresent(RequiresInstanceClassLoading.class);
-
-        ConfigurableComponentInitializer initializer = null;
-        try {
-            initializer = ConfigurableComponentInitializerFactory.createComponentInitializer(component.getClass());
-            initializer.initialize(component);
-
-            final Set<Class> cobundledApis = new HashSet<>();
-            try (final NarCloseable closeable = NarCloseable.withComponentNarLoader(component.getClass().getClassLoader())) {
-                final List<PropertyDescriptor> descriptors = component.getPropertyDescriptors();
-                if (descriptors != null && !descriptors.isEmpty()) {
-                    for (final PropertyDescriptor descriptor : descriptors) {
-                        final Class<? extends ControllerService> serviceApi = descriptor.getControllerServiceDefinition();
-                        if (serviceApi != null && classLoader.equals(serviceApi.getClassLoader())) {
-                            cobundledApis.add(serviceApi);
-                        }
-                    }
-                }
-            }
-
-            if (!cobundledApis.isEmpty()) {
-                logger.warn(String.format(
-                        "Component %s is bundled with its referenced Controller Service APIs %s. The service APIs should not be bundled with component implementations that reference it.",
-                        component.getClass().getName(), StringUtils.join(cobundledApis.stream().map(cls -> cls.getName()).collect(Collectors.toSet()), ", ")));
-            }
-
-            // the component is eligible when it does not require instance classloading or when the supporting APIs are bundled in a parent NAR
-            return requiresInstanceClassLoading == false || cobundledApis.isEmpty();
-        } catch (final InitializationException e) {
-            logger.warn(String.format("Unable to verify if component %s references any bundled Controller Service APIs due to %s", component.getClass().getName(), e.getMessage()));
-            return true;
-        } finally {
-            if (initializer != null) {
-                initializer.teardown(component);
-            }
-        }
-    }
-
-    private static boolean checkControllerServiceEligibility(Class extensionType) {
-        final Class originalExtensionType = extensionType;
-        final ClassLoader originalExtensionClassLoader = extensionType.getClassLoader();
-
-        // if the extension does not require instance classloading, its eligible
-        final boolean requiresInstanceClassLoading = extensionType.isAnnotationPresent(RequiresInstanceClassLoading.class);
-
-        final Set<Class> cobundledApis = new HashSet<>();
-        while (extensionType != null) {
-            for (final Class i : extensionType.getInterfaces()) {
-                if (originalExtensionClassLoader.equals(i.getClassLoader())) {
-                    cobundledApis.add(i);
-                }
-            }
-
-            extensionType = extensionType.getSuperclass();
-        }
-
-        if (!cobundledApis.isEmpty()) {
-            logger.warn(String.format("Controller Service %s is bundled with its supporting APIs %s. The service APIs should not be bundled with the implementations.",
-                    originalExtensionType.getName(), StringUtils.join(cobundledApis.stream().map(cls -> cls.getName()).collect(Collectors.toSet()), ", ")));
-        }
-
-        // the service is eligible when it does not require instance classloading or when the supporting APIs are bundled in a parent NAR
-        return requiresInstanceClassLoading == false || cobundledApis.isEmpty();
-    }
-
-    /**
-     * Registers extension for the specified type from the specified Bundle.
-     *
-     * @param type the extension type
-     * @param classNameBundleMap mapping of classname to Bundle
-     * @param bundle the Bundle being mapped to
-     * @param classes to map to this classloader but which come from its ancestors
-     */
-    private static void registerServiceClass(final Class<?> type, final Map<String, List<Bundle>> classNameBundleMap, final Bundle bundle, final Set<Class> classes) {
-        final String className = type.getName();
-
-        // get the bundles that have already been registered for the class name
-        List<Bundle> registeredBundles = classNameBundleMap.get(className);
-
-        if (registeredBundles == null) {
-            registeredBundles = new ArrayList<>();
-            classNameBundleMap.put(className, registeredBundles);
-        }
-
-        boolean alreadyRegistered = false;
-        for (final Bundle registeredBundle : registeredBundles) {
-            final BundleCoordinate registeredCoordinate = registeredBundle.getBundleDetails().getCoordinate();
-
-            // if the incoming bundle has the same coordinate as one of the registered bundles then consider it already registered
-            if (registeredCoordinate.equals(bundle.getBundleDetails().getCoordinate())) {
-                alreadyRegistered = true;
-                break;
-            }
-
-            // if the type wasn't loaded from an ancestor, and the type isn't a processor, cs, or reporting task, then
-            // fail registration because we don't support multiple versions of any other types
-            if (!multipleVersionsAllowed(type)) {
-                throw new IllegalStateException("Attempt was made to load " + className + " from "
-                        + bundle.getBundleDetails().getCoordinate().getCoordinate()
-                        + " but that class name is already loaded/registered from " + registeredBundle.getBundleDetails().getCoordinate()
-                        + " and multiple versions are not supported for this type"
-                );
-            }
-        }
-
-        // if none of the above was true then register the new bundle
-        if (!alreadyRegistered) {
-            registeredBundles.add(bundle);
-            classes.add(type);
-
-            if (type.isAnnotationPresent(RequiresInstanceClassLoading.class)) {
-                requiresInstanceClassLoading.add(className);
-            }
-        }
-
-    }
-
-    /**
-     * @param type a Class that we found from a service loader
-     * @return true if the given class is a processor, controller service, or reporting task
-     */
-    private static boolean multipleVersionsAllowed(Class<?> type) {
-        return Processor.class.isAssignableFrom(type) || ControllerService.class.isAssignableFrom(type) || ReportingTask.class.isAssignableFrom(type);
-    }
-
-    /**
-     * Determines the effective ClassLoader for the instance of the given type.
-     *
-     * @param classType the type of class to lookup the ClassLoader for
-     * @param instanceIdentifier the identifier of the specific instance of the classType to look up the ClassLoader for
-     * @param bundle the bundle where the classType exists
-     * @return the ClassLoader for the given instance of the given type, or null if the type is not a detected extension type
-     */
-    public static ClassLoader createInstanceClassLoader(final String classType, final String instanceIdentifier, final Bundle bundle) {
-        if (StringUtils.isEmpty(classType)) {
-            throw new IllegalArgumentException("Class-Type is required");
-        }
-
-        if (StringUtils.isEmpty(instanceIdentifier)) {
-            throw new IllegalArgumentException("Instance Identifier is required");
-        }
-
-        if (bundle == null) {
-            throw new IllegalArgumentException("Bundle is required");
-        }
-
-        final ClassLoader bundleClassLoader = bundle.getClassLoader();
-
-        // If the class is annotated with @RequiresInstanceClassLoading and the registered ClassLoader is a URLClassLoader
-        // then make a new InstanceClassLoader that is a full copy of the NAR Class Loader, otherwise create an empty
-        // InstanceClassLoader that has the NAR ClassLoader as a parent
-        ClassLoader instanceClassLoader;
-        if (requiresInstanceClassLoading.contains(classType) && (bundleClassLoader instanceof URLClassLoader)) {
-            final URLClassLoader registeredUrlClassLoader = (URLClassLoader) bundleClassLoader;
-            instanceClassLoader = new InstanceClassLoader(instanceIdentifier, classType, registeredUrlClassLoader.getURLs(), registeredUrlClassLoader.getParent());
-        } else {
-            instanceClassLoader = new InstanceClassLoader(instanceIdentifier, classType, new URL[0], bundleClassLoader);
-        }
-
-        instanceClassloaderLookup.put(instanceIdentifier, instanceClassLoader);
-        return instanceClassLoader;
-    }
-
-    /**
-     * Retrieves the InstanceClassLoader for the component with the given identifier.
-     *
-     * @param instanceIdentifier the identifier of a component
-     * @return the instance class loader for the component
-     */
-    public static ClassLoader getInstanceClassLoader(final String instanceIdentifier) {
-        return instanceClassloaderLookup.get(instanceIdentifier);
-    }
-
-    /**
-     * Removes the ClassLoader for the given instance and closes it if necessary.
-     *
-     * @param instanceIdentifier the identifier of a component to remove the ClassLoader for
-     * @return the removed ClassLoader for the given instance, or null if not found
-     */
-    public static ClassLoader removeInstanceClassLoaderIfExists(final String instanceIdentifier) {
-        if (instanceIdentifier == null) {
-            return null;
-        }
-
-        final ClassLoader classLoader = instanceClassloaderLookup.remove(instanceIdentifier);
-        if (classLoader != null && (classLoader instanceof URLClassLoader)) {
-            final URLClassLoader urlClassLoader = (URLClassLoader) classLoader;
-            try {
-                urlClassLoader.close();
-            } catch (IOException e) {
-                logger.warn("Unable to class URLClassLoader for " + instanceIdentifier);
-            }
-        }
-        return classLoader;
-    }
-
-    /**
-     * Checks if the given class type requires per-instance class loading (i.e. contains the @RequiresInstanceClassLoading annotation)
-     *
-     * @param classType the class to check
-     * @return true if the class is found in the set of classes requiring instance level class loading, false otherwise
-     */
-    public static boolean requiresInstanceClassLoading(final String classType) {
-        if (classType == null) {
-            throw new IllegalArgumentException("Class type cannot be null");
-        }
-        return requiresInstanceClassLoading.contains(classType);
-    }
-
-    /**
-     * Retrieves the bundles that have a class with the given name.
-     *
-     * @param classType the class name of an extension
-     * @return the list of bundles that contain an extension with the given class name
-     */
-    public static List<Bundle> getBundles(final String classType) {
-        if (classType == null) {
-            throw new IllegalArgumentException("Class type cannot be null");
-        }
-        final List<Bundle> bundles = classNameBundleLookup.get(classType);
-        return bundles == null ? Collections.emptyList() : new ArrayList<>(bundles);
-    }
-
-    /**
-     * Retrieves the bundle with the given coordinate.
-     *
-     * @param bundleCoordinate a coordinate to look up
-     * @return the bundle with the given coordinate, or null if none exists
-     */
-    public static Bundle getBundle(final BundleCoordinate bundleCoordinate) {
-        if (bundleCoordinate == null) {
-            throw new IllegalArgumentException("BundleCoordinate cannot be null");
-        }
-        return bundleCoordinateBundleLookup.get(bundleCoordinate);
-    }
-
-    /**
-     * Retrieves the bundle for the given class loader.
-     *
-     * @param classLoader the class loader to look up the bundle for
-     * @return the bundle for the given class loader
-     */
-    public static Bundle getBundle(final ClassLoader classLoader) {
-        if (classLoader == null) {
-            throw new IllegalArgumentException("ClassLoader cannot be null");
-        }
-        return classLoaderBundleLookup.get(classLoader);
-    }
-
-    public static Set<Class> getExtensions(final Class<?> definition) {
-        if (definition == null) {
-            throw new IllegalArgumentException("Class cannot be null");
-        }
-        final Set<Class> extensions = definitionMap.get(definition);
-        return (extensions == null) ? Collections.<Class>emptySet() : extensions;
-    }
-
-    public static void logClassLoaderMapping() {
-        final StringBuilder builder = new StringBuilder();
-
-        builder.append("Extension Type Mapping to Bundle:");
-        for (final Map.Entry<Class, Set<Class>> entry : definitionMap.entrySet()) {
-            builder.append("\n\t=== ").append(entry.getKey().getSimpleName()).append(" Type ===");
-
-            for (final Class type : entry.getValue()) {
-                final List<Bundle> bundles = classNameBundleLookup.containsKey(type.getName())
-                        ? classNameBundleLookup.get(type.getName()) : Collections.emptyList();
-
-                builder.append("\n\t").append(type.getName());
-
-                for (final Bundle bundle : bundles) {
-                    final String coordinate = bundle.getBundleDetails().getCoordinate().getCoordinate();
-                    final String workingDir = bundle.getBundleDetails().getWorkingDirectory().getPath();
-                    builder.append("\n\t\t").append(coordinate).append(" || ").append(workingDir);
-                }
-            }
-
-            builder.append("\n\t=== End ").append(entry.getKey().getSimpleName()).append(" types ===");
-        }
-
-        logger.info(builder.toString());
-    }
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/556f309d/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/nar/InstanceClassLoader.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/nar/InstanceClassLoader.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/nar/InstanceClassLoader.java
deleted file mode 100644
index 2a9c72d..0000000
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/nar/InstanceClassLoader.java
+++ /dev/null
@@ -1,160 +0,0 @@
-/*
- * 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 org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.net.URL;
-import java.net.URLClassLoader;
-
-/**
- * A ClassLoader created for an instance of a component which lets a client add resources to an intermediary ClassLoader
- * that will be checked first when loading/finding classes.
- *
- * Typically an instance of this ClassLoader will be created by passing in the URLs and parent from a NARClassLoader in
- * order to create a copy of the NARClassLoader without modifying it.
- */
-public class InstanceClassLoader extends URLClassLoader {
-
-    private static final Logger logger = LoggerFactory.getLogger(InstanceClassLoader.class);
-
-    private final String identifier;
-    private final String instanceType;
-    private ShimClassLoader shimClassLoader;
-
-    /**
-     * @param identifier the id of the component this ClassLoader was created for
-     * @param urls the URLs for the ClassLoader
-     * @param parent the parent ClassLoader
-     */
-    public InstanceClassLoader(final String identifier, final String type, final URL[] urls, final ClassLoader parent) {
-        super(urls, parent);
-        this.identifier = identifier;
-        this.instanceType = type;
-    }
-
-    /**
-     * Initializes a new ShimClassLoader for the provided resources, closing the previous ShimClassLoader if one existed.
-     *
-     * @param urls the URLs for the ShimClassLoader
-     * @throws IOException if the previous ShimClassLoader existed and couldn't be closed
-     */
-    public synchronized void setInstanceResources(final URL[] urls) {
-        if (shimClassLoader != null) {
-            try {
-                shimClassLoader.close();
-            } catch (IOException e) {
-                logger.warn("Unable to close inner URLClassLoader for " + identifier);
-            }
-        }
-
-        shimClassLoader = new ShimClassLoader(urls, getParent());
-    }
-
-    /**
-     * @return the URLs for the instance resources that have been set
-     */
-    public synchronized URL[] getInstanceResources() {
-        if (shimClassLoader != null) {
-            return shimClassLoader.getURLs();
-        }
-        return new URL[0];
-    }
-
-    @Override
-    public Class<?> loadClass(String name) throws ClassNotFoundException {
-        return this.loadClass(name, false);
-    }
-
-    @Override
-    protected Class<?> loadClass(String name, boolean resolve) throws ClassNotFoundException {
-        Class<?> c = null;
-        // first try the shim
-        if (shimClassLoader != null) {
-            try {
-                c = shimClassLoader.loadClass(name, resolve);
-            } catch (ClassNotFoundException e) {
-                c = null;
-            }
-        }
-        // if it wasn't in the shim try our self
-        if (c == null) {
-            return super.loadClass(name, resolve);
-        } else {
-            return c;
-        }
-    }
-
-    @Override
-    protected Class<?> findClass(String name) throws ClassNotFoundException {
-        Class<?> c = null;
-        // first try the shim
-        if (shimClassLoader != null) {
-            try {
-                c = shimClassLoader.findClass(name);
-            } catch (ClassNotFoundException cnf) {
-                c = null;
-            }
-        }
-        // if it wasn't in the shim try our self
-        if (c == null) {
-            return super.findClass(name);
-        } else {
-            return c;
-        }
-    }
-
-    @Override
-    public void close() throws IOException {
-        if (shimClassLoader != null) {
-            try {
-                shimClassLoader.close();
-            } catch (IOException e) {
-                logger.warn("Unable to close inner URLClassLoader for " + identifier);
-            }
-        }
-        super.close();
-    }
-
-    /**
-     * Extend URLClassLoader to increase visibility of protected methods so that InstanceClassLoader can delegate.
-     */
-    private static class ShimClassLoader extends URLClassLoader {
-
-        public ShimClassLoader(URL[] urls, ClassLoader parent) {
-            super(urls, parent);
-        }
-
-        public ShimClassLoader(URL[] urls) {
-            super(urls);
-        }
-
-        @Override
-        public Class<?> findClass(String name) throws ClassNotFoundException {
-            return super.findClass(name);
-        }
-
-        @Override
-        public Class<?> loadClass(String name, boolean resolve) throws ClassNotFoundException {
-            return super.loadClass(name, resolve);
-        }
-
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/556f309d/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 a656e76..005a8fa 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
@@ -184,6 +184,7 @@ public final class NarClassLoaders {
                     jettyClassLoader = createNarClassLoader(narDetail.getWorkingDirectory(), systemClassLoader);
 
                     // remove the jetty nar since its already loaded
+                    narDirectoryBundleLookup.put(narDetail.getWorkingDirectory().getCanonicalPath(), new Bundle(narDetail, jettyClassLoader));
                     narCoordinateClassLoaderLookup.put(narDetail.getCoordinate().getCoordinate(), jettyClassLoader);
                     narDetailsIter.remove();
                 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/556f309d/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
deleted file mode 100644
index 88d47ff..0000000
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/nar/NarCloseable.java
+++ /dev/null
@@ -1,112 +0,0 @@
-/*
- * 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 java.io.Closeable;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- *
- */
-public class NarCloseable implements Closeable {
-
-    private static final Logger logger = LoggerFactory.getLogger(NarCloseable.class);
-
-    public static NarCloseable withNarLoader() {
-        final ClassLoader current = Thread.currentThread().getContextClassLoader();
-        Thread.currentThread().setContextClassLoader(NarThreadContextClassLoader.getInstance());
-        return new NarCloseable(current);
-    }
-
-    /**
-     * Sets the current thread context class loader to the specific appropriate class loader for the given
-     * component. If the component requires per-instance class loading then the class loader will be the
-     * specific class loader for instance with the given identifier, otherwise the class loader will be
-     * the NARClassLoader.
-     *
-     * @param componentClass the component class
-     * @param componentIdentifier the identifier of the component
-     * @return NarCloseable with the current thread context classloader jailed to the Nar
-     *              or instance class loader of the component
-     */
-    public static NarCloseable withComponentNarLoader(final Class componentClass, final String componentIdentifier) {
-        final ClassLoader current = Thread.currentThread().getContextClassLoader();
-
-        ClassLoader componentClassLoader = ExtensionManager.getInstanceClassLoader(componentIdentifier);
-        if (componentClassLoader == null) {
-            componentClassLoader = componentClass.getClassLoader();
-        }
-
-        Thread.currentThread().setContextClassLoader(componentClassLoader);
-        return new NarCloseable(current);
-    }
-
-    /**
-     * 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
-     *
-     * @return a NarCloseable
-     */
-    public static NarCloseable withFrameworkNar() {
-        final ClassLoader frameworkClassLoader;
-        try {
-            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.");
-            if (logger.isDebugEnabled()) {
-                logger.error("", e);
-            }
-
-            return new NarCloseable(null);
-        }
-
-        final ClassLoader current = Thread.currentThread().getContextClassLoader();
-        Thread.currentThread().setContextClassLoader(frameworkClassLoader);
-        return new NarCloseable(current);
-    }
-
-    private final ClassLoader toSet;
-
-    private NarCloseable(final ClassLoader toSet) {
-        this.toSet = toSet;
-    }
-
-    @Override
-    public void close() {
-        if (toSet != null) {
-            Thread.currentThread().setContextClassLoader(toSet);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/556f309d/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
index da54c4e..8b02742 100644
--- 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
@@ -32,7 +32,7 @@ public enum NarManifestEntry {
     BUILD_BRANCH("Build-Branch"),
     BUILD_TIMESTAMP("Build-Timestamp"),
     BUILD_JDK("Build-Jdk"),
-    BUILT_BY("Built-By")
+    BUILT_BY("Built-By"),
     ;
 
     final String manifestName;

http://git-wip-us.apache.org/repos/asf/nifi/blob/556f309d/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
deleted file mode 100644
index e7faa02..0000000
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/nar/NarThreadContextClassLoader.java
+++ /dev/null
@@ -1,235 +0,0 @@
-/*
- * 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 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;
-import org.apache.nifi.controller.repository.FlowFileSwapManager;
-import org.apache.nifi.controller.status.history.ComponentStatusRepository;
-import org.apache.nifi.flowfile.FlowFilePrioritizer;
-import org.apache.nifi.processor.Processor;
-import org.apache.nifi.processor.io.InputStreamCallback;
-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;
-import java.lang.reflect.Constructor;
-import java.lang.reflect.InvocationTargetException;
-import java.net.URL;
-import java.net.URLClassLoader;
-import java.util.ArrayList;
-import java.util.Enumeration;
-import java.util.List;
-
-/**
- * THREAD SAFE
- */
-public class NarThreadContextClassLoader extends URLClassLoader {
-
-    static final ContextSecurityManager contextSecurityManager = new ContextSecurityManager();
-    private final ClassLoader forward = ClassLoader.getSystemClassLoader();
-    private static final List<Class<?>> narSpecificClasses = new ArrayList<>();
-
-    static {
-        narSpecificClasses.add(Processor.class);
-        narSpecificClasses.add(FlowFilePrioritizer.class);
-        narSpecificClasses.add(ReportingTask.class);
-        narSpecificClasses.add(Validator.class);
-        narSpecificClasses.add(InputStreamCallback.class);
-        narSpecificClasses.add(OutputStreamCallback.class);
-        narSpecificClasses.add(StreamCallback.class);
-        narSpecificClasses.add(ControllerService.class);
-        narSpecificClasses.add(Authorizer.class);
-        narSpecificClasses.add(LoginIdentityProvider.class);
-        narSpecificClasses.add(ProvenanceRepository.class);
-        narSpecificClasses.add(ComponentStatusRepository.class);
-        narSpecificClasses.add(FlowFileRepository.class);
-        narSpecificClasses.add(FlowFileSwapManager.class);
-        narSpecificClasses.add(ContentRepository.class);
-        narSpecificClasses.add(StateProvider.class);
-    }
-
-    private NarThreadContextClassLoader() {
-        super(new URL[0]);
-    }
-
-    @Override
-    public void clearAssertionStatus() {
-        lookupClassLoader().clearAssertionStatus();
-    }
-
-    @Override
-    public URL getResource(String name) {
-        return lookupClassLoader().getResource(name);
-    }
-
-    @Override
-    public InputStream getResourceAsStream(String name) {
-        return lookupClassLoader().getResourceAsStream(name);
-    }
-
-    @Override
-    public Enumeration<URL> getResources(String name) throws IOException {
-        return lookupClassLoader().getResources(name);
-    }
-
-    @Override
-    public Class<?> loadClass(String name) throws ClassNotFoundException {
-        return lookupClassLoader().loadClass(name);
-    }
-
-    @Override
-    public void setClassAssertionStatus(String className, boolean enabled) {
-        lookupClassLoader().setClassAssertionStatus(className, enabled);
-    }
-
-    @Override
-    public void setDefaultAssertionStatus(boolean enabled) {
-        lookupClassLoader().setDefaultAssertionStatus(enabled);
-    }
-
-    @Override
-    public void setPackageAssertionStatus(String packageName, boolean enabled) {
-        lookupClassLoader().setPackageAssertionStatus(packageName, enabled);
-    }
-
-    private ClassLoader lookupClassLoader() {
-        final Class<?>[] classStack = contextSecurityManager.getExecutionStack();
-
-        for (Class<?> currentClass : classStack) {
-            final Class<?> narClass = findNarClass(currentClass);
-            if (narClass != null) {
-                final ClassLoader desiredClassLoader = narClass.getClassLoader();
-
-                // When new Threads are created, the new Thread inherits the ClassLoaderContext of
-                // the caller. However, the call stack of that new Thread may not trace back to any NiFi-specific
-                // code. Therefore, the NarThreadContextClassLoader will be unable to find the appropriate NAR
-                // ClassLoader. As a result, we want to set the ContextClassLoader to the NAR ClassLoader that
-                // contains the class or resource that we are looking for.
-                // This locks the current Thread into the appropriate NAR ClassLoader Context. The framework will change
-                // the ContextClassLoader back to the NarThreadContextClassLoader as appropriate via the
-                // {@link FlowEngine.beforeExecute(Thread, Runnable)} and
-                // {@link FlowEngine.afterExecute(Thread, Runnable)} methods.
-                if (desiredClassLoader instanceof NarClassLoader) {
-                    Thread.currentThread().setContextClassLoader(desiredClassLoader);
-                }
-                return desiredClassLoader;
-            }
-        }
-        return forward;
-    }
-
-    private Class<?> findNarClass(final Class<?> cls) {
-        for (final Class<?> narClass : narSpecificClasses) {
-            if (narClass.isAssignableFrom(cls)) {
-                return cls;
-            } else if (cls.getEnclosingClass() != null) {
-                return findNarClass(cls.getEnclosingClass());
-            }
-        }
-
-        return null;
-    }
-
-    private static class SingletonHolder {
-
-        public static final NarThreadContextClassLoader instance = new NarThreadContextClassLoader();
-    }
-
-    public static NarThreadContextClassLoader getInstance() {
-        return SingletonHolder.instance;
-    }
-
-    static class ContextSecurityManager extends SecurityManager {
-
-        Class<?>[] getExecutionStack() {
-            return getClassContext();
-        }
-    }
-
-    /**
-     * Constructs an instance of the given type using either default no args
-     * constructor or a constructor which takes a NiFiProperties object
-     * (preferred).
-     *
-     * @param <T> the type to create an instance for
-     * @param implementationClassName the implementation class name
-     * @param typeDefinition the type definition
-     * @param nifiProperties the NiFiProperties instance
-     * @return constructed instance
-     * @throws InstantiationException if there is an error instantiating the class
-     * @throws IllegalAccessException if there is an error accessing the type
-     * @throws ClassNotFoundException if the class cannot be found
-     */
-    public static <T> T createInstance(final String implementationClassName, final Class<T> typeDefinition, final NiFiProperties nifiProperties)
-            throws InstantiationException, IllegalAccessException, ClassNotFoundException {
-        final ClassLoader originalClassLoader = Thread.currentThread().getContextClassLoader();
-        Thread.currentThread().setContextClassLoader(NarThreadContextClassLoader.getInstance());
-        try {
-            final List<Bundle> bundles = ExtensionManager.getBundles(implementationClassName);
-            if (bundles.size() == 0) {
-                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);
-            if(nifiProperties == null){
-                return typeDefinition.cast(desiredClass.newInstance());
-            }
-            Constructor<?> constructor = null;
-
-            try {
-                constructor = desiredClass.getConstructor(NiFiProperties.class);
-            } catch (NoSuchMethodException nsme) {
-                try {
-                    constructor = desiredClass.getConstructor();
-                } catch (NoSuchMethodException nsme2) {
-                    throw new IllegalStateException("Failed to find constructor which takes NiFiProperties as argument as well as the default constructor on "
-                            + desiredClass.getName(), nsme2);
-                }
-            }
-            try {
-                if (constructor.getParameterTypes().length == 0) {
-                    return typeDefinition.cast(constructor.newInstance());
-                } else {
-                    return typeDefinition.cast(constructor.newInstance(nifiProperties));
-                }
-            } catch (InvocationTargetException ite) {
-                throw new IllegalStateException("Failed to instantiate a component due to (see target exception)", ite);
-            }
-        } finally {
-            Thread.currentThread().setContextClassLoader(originalClassLoader);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/556f309d/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/nar/SystemBundle.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/nar/SystemBundle.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/nar/SystemBundle.java
new file mode 100644
index 0000000..0fb2bad
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/nar/SystemBundle.java
@@ -0,0 +1,57 @@
+/*
+ * 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 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.apache.nifi.util.StringUtils;
+
+import java.io.File;
+
+/**
+ * Utility to create the system bundle.
+ */
+public final class SystemBundle {
+
+    public static final BundleCoordinate SYSTEM_BUNDLE_COORDINATE = new BundleCoordinate(
+            BundleCoordinate.DEFAULT_GROUP, "system", BundleCoordinate.DEFAULT_VERSION);
+
+    /**
+     * Returns a bundle representing the system class loader.
+     *
+     * @param niFiProperties a NiFiProperties instance which will be used to obtain the default NAR library path,
+     *                       which will become the working directory of the returned bundle
+     * @return a bundle for the system class loader
+     */
+    public static Bundle create(final NiFiProperties niFiProperties) {
+        final ClassLoader systemClassLoader = ClassLoader.getSystemClassLoader();
+
+        final String narLibraryDirectory = niFiProperties.getProperty(NiFiProperties.NAR_LIBRARY_DIRECTORY);
+        if (StringUtils.isBlank(narLibraryDirectory)) {
+            throw new IllegalStateException("Unable to create system bundle because " + NiFiProperties.NAR_LIBRARY_DIRECTORY + " was null or empty");
+        }
+
+        final BundleDetails systemBundleDetails = new BundleDetails.Builder()
+                .workingDir(new File(narLibraryDirectory))
+                .coordinate(SYSTEM_BUNDLE_COORDINATE)
+                .build();
+
+        return new Bundle(systemBundleDetails, systemClassLoader);
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/556f309d/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
index aa526c6..df6a361 100644
--- 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
@@ -16,9 +16,6 @@
  */
 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;
@@ -26,6 +23,9 @@ import org.junit.Test;
 import java.io.File;
 import java.io.IOException;
 
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+
 public class NarBundleUtilTest {
 
     @Test

http://git-wip-us.apache.org/repos/asf/nifi/blob/556f309d/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
deleted file mode 100644
index 4528c0a..0000000
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/test/java/org/apache/nifi/nar/NarThreadContextClassLoaderTest.java
+++ /dev/null
@@ -1,99 +0,0 @@
-/*
- * 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.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;
-
-public class NarThreadContextClassLoaderTest {
-
-    @Test
-    public void validateWithPropertiesConstructor() throws Exception {
-        NiFiProperties properties = NiFiProperties.createBasicNiFiProperties("src/test/resources/nifi.properties", null);
-        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)
-    public void validateWithPropertiesConstructorInstantiationFailure() throws Exception {
-        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);
-    }
-
-    public static class WithPropertiesConstructor extends AbstractProcessor {
-        private NiFiProperties properties;
-
-        public 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 extends AbstractProcessor {
-        public WithDefaultConstructor() {
-
-        }
-
-        @Override
-        public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException {
-
-        }
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/556f309d/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
deleted file mode 100644
index 8cc2ccd..0000000
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/test/java/org/apache/nifi/nar/NarUnpackerTest.java
+++ /dev/null
@@ -1,183 +0,0 @@
-/*
- * 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 org.apache.nifi.util.NiFiProperties;
-import org.junit.Assert;
-import org.junit.BeforeClass;
-import org.junit.Test;
-
-import java.io.File;
-import java.io.IOException;
-import java.net.URISyntaxException;
-import java.nio.file.FileVisitResult;
-import java.nio.file.Files;
-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 static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertTrue;
-
-public class NarUnpackerTest {
-
-    @BeforeClass
-    public static void copyResources() throws IOException {
-
-        final Path sourcePath = Paths.get("./src/test/resources");
-        final Path targetPath = Paths.get("./target");
-
-        Files.walkFileTree(sourcePath, new SimpleFileVisitor<Path>() {
-
-            @Override
-            public FileVisitResult preVisitDirectory(Path dir, BasicFileAttributes attrs)
-                    throws IOException {
-
-                Path relativeSource = sourcePath.relativize(dir);
-                Path target = targetPath.resolve(relativeSource);
-
-                Files.createDirectories(target);
-
-                return FileVisitResult.CONTINUE;
-
-            }
-
-            @Override
-            public FileVisitResult visitFile(Path file, BasicFileAttributes attrs)
-                    throws IOException {
-
-                Path relativeSource = sourcePath.relativize(file);
-                Path target = targetPath.resolve(relativeSource);
-
-                Files.copy(file, target, REPLACE_EXISTING);
-
-                return FileVisitResult.CONTINUE;
-            }
-        });
-    }
-
-    @Test
-    public void testUnpackNars() {
-
-        NiFiProperties properties = loadSpecifiedProperties("/NarUnpacker/conf/nifi.properties", Collections.EMPTY_MAP);
-
-        assertEquals("./target/NarUnpacker/lib/",
-                properties.getProperty("nifi.nar.library.directory"));
-        assertEquals("./target/NarUnpacker/lib2/",
-                properties.getProperty("nifi.nar.library.directory.alt"));
-
-        final ExtensionMapping extensionMapping = NarUnpacker.unpackNars(properties, ExtensionManager.createSystemBundle(properties));
-
-        assertEquals(2, extensionMapping.getAllExtensionNames().size());
-
-        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();
-
-        Set<String> expectedNars = new HashSet<>();
-        expectedNars.add("dummy-one.nar-unpacked");
-        expectedNars.add("dummy-two.nar-unpacked");
-        assertEquals(expectedNars.size(), extensionFiles.length);
-
-        for (File extensionFile : extensionFiles) {
-            Assert.assertTrue(expectedNars.contains(extensionFile.getName()));
-        }
-    }
-
-    @Test
-    public void testUnpackNarsFromEmptyDir() throws IOException {
-
-        final File emptyDir = new File("./target/empty/dir");
-        emptyDir.delete();
-        emptyDir.deleteOnExit();
-        assertTrue(emptyDir.mkdirs());
-
-        final Map<String, String> others = new HashMap<>();
-        others.put("nifi.nar.library.directory.alt", emptyDir.toString());
-        NiFiProperties properties = loadSpecifiedProperties("/NarUnpacker/conf/nifi.properties", others);
-
-        final ExtensionMapping extensionMapping = NarUnpacker.unpackNars(properties, ExtensionManager.createSystemBundle(properties));
-
-        assertEquals(1, extensionMapping.getAllExtensionNames().size());
-        assertTrue(extensionMapping.getAllExtensionNames().keySet().contains("org.apache.nifi.processors.dummy.one"));
-
-        final File extensionsWorkingDir = properties.getExtensionsWorkingDirectory();
-        File[] extensionFiles = extensionsWorkingDir.listFiles();
-
-        assertEquals(1, extensionFiles.length);
-        assertEquals("dummy-one.nar-unpacked", extensionFiles[0].getName());
-    }
-
-    @Test
-    public void testUnpackNarsFromNonExistantDir() {
-
-        final File nonExistantDir = new File("./target/this/dir/should/not/exist/");
-        nonExistantDir.delete();
-        nonExistantDir.deleteOnExit();
-
-        final Map<String, String> others = new HashMap<>();
-        others.put("nifi.nar.library.directory.alt", nonExistantDir.toString());
-        NiFiProperties properties = loadSpecifiedProperties("/NarUnpacker/conf/nifi.properties", others);
-
-        final ExtensionMapping extensionMapping = NarUnpacker.unpackNars(properties, ExtensionManager.createSystemBundle(properties));
-
-        assertTrue(extensionMapping.getAllExtensionNames().keySet().contains("org.apache.nifi.processors.dummy.one"));
-
-        assertEquals(1, extensionMapping.getAllExtensionNames().size());
-
-        final File extensionsWorkingDir = properties.getExtensionsWorkingDirectory();
-        File[] extensionFiles = extensionsWorkingDir.listFiles();
-
-        assertEquals(1, extensionFiles.length);
-        assertEquals("dummy-one.nar-unpacked", extensionFiles[0].getName());
-    }
-
-    @Test
-    public void testUnpackNarsFromNonDir() throws IOException {
-
-        final File nonDir = new File("./target/file.txt");
-        nonDir.createNewFile();
-        nonDir.deleteOnExit();
-
-        final Map<String, String> others = new HashMap<>();
-        others.put("nifi.nar.library.directory.alt", nonDir.toString());
-        NiFiProperties properties = loadSpecifiedProperties("/NarUnpacker/conf/nifi.properties", others);
-
-        final ExtensionMapping extensionMapping = NarUnpacker.unpackNars(properties, ExtensionManager.createSystemBundle(properties));
-
-        assertNull(extensionMapping);
-    }
-
-    private NiFiProperties loadSpecifiedProperties(final String propertiesFile, final Map<String, String> others) {
-        String filePath;
-        try {
-            filePath = NarUnpackerTest.class.getResource(propertiesFile).toURI().getPath();
-        } catch (URISyntaxException 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/556f309d/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
deleted file mode 100644
index 9d180b6..0000000
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/test/resources/META-INF/services/org.apache.nifi.processor.Processor
+++ /dev/null
@@ -1,16 +0,0 @@
-# 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/556f309d/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
deleted file mode 100644
index 0559752..0000000
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/test/resources/NarUnpacker/conf/nifi.properties
+++ /dev/null
@@ -1,124 +0,0 @@
-# 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.
-
-# Core Properties #
-nifi.flow.configuration.file=./target/flow.xml.gz
-nifi.flow.configuration.archive.dir=./target/archive/
-nifi.flowcontroller.autoResumeState=true
-nifi.flowcontroller.graceful.shutdown.period=10 sec
-nifi.flowservice.writedelay.interval=2 sec
-nifi.administrative.yield.duration=30 sec
-
-nifi.reporting.task.configuration.file=./target/reporting-tasks.xml
-nifi.controller.service.configuration.file=./target/controller-services.xml
-nifi.templates.directory=./target/templates
-nifi.ui.banner.text=UI Banner Text
-nifi.ui.autorefresh.interval=30 sec
-nifi.nar.library.directory=./target/NarUnpacker/lib/
-nifi.nar.library.directory.alt=./target/NarUnpacker/lib2/
-
-nifi.nar.working.directory=./target/work/nar/
-
-# H2 Settings
-nifi.database.directory=./target/database_repository
-nifi.h2.url.append=;LOCK_TIMEOUT=25000;WRITE_DELAY=0;AUTO_SERVER=FALSE
-
-# FlowFile Repository
-nifi.flowfile.repository.directory=./target/test-repo
-nifi.flowfile.repository.partitions=1
-nifi.flowfile.repository.checkpoint.interval=2 mins
-nifi.queue.swap.threshold=20000
-nifi.swap.storage.directory=./target/test-repo/swap
-nifi.swap.in.period=5 sec
-nifi.swap.in.threads=1
-nifi.swap.out.period=5 sec
-nifi.swap.out.threads=4
-
-# Content Repository
-nifi.content.claim.max.appendable.size=10 MB
-nifi.content.claim.max.flow.files=100
-nifi.content.repository.directory.default=./target/content_repository
-
-# Provenance Repository Properties
-nifi.provenance.repository.storage.directory=./target/provenance_repository
-nifi.provenance.repository.max.storage.time=24 hours
-nifi.provenance.repository.max.storage.size=1 GB
-nifi.provenance.repository.rollover.time=30 secs
-nifi.provenance.repository.rollover.size=100 MB
-
-# Site to Site properties
-nifi.remote.input.socket.port=9990
-nifi.remote.input.secure=true
-
-# web properties #
-nifi.web.war.directory=./target/lib
-nifi.web.http.host=
-nifi.web.http.port=8080
-nifi.web.https.host=
-nifi.web.https.port=
-nifi.web.jetty.working.directory=./target/work/jetty
-
-# security properties #
-nifi.sensitive.props.key=key
-nifi.sensitive.props.algorithm=PBEWITHMD5AND256BITAES-CBC-OPENSSL
-nifi.sensitive.props.provider=BC
-
-nifi.security.keystore=
-nifi.security.keystoreType=
-nifi.security.keystorePasswd=
-nifi.security.keyPasswd=
-nifi.security.truststore=
-nifi.security.truststoreType=
-nifi.security.truststorePasswd=
-nifi.security.needClientAuth=
-nifi.security.user.authorizer=
-
-# cluster common properties (cluster manager and nodes must have same values) #
-nifi.cluster.protocol.heartbeat.interval=5 sec
-nifi.cluster.protocol.is.secure=false
-nifi.cluster.protocol.socket.timeout=30 sec
-nifi.cluster.protocol.connection.handshake.timeout=45 sec
-# if multicast is used, then nifi.cluster.protocol.multicast.xxx properties must be configured #
-nifi.cluster.protocol.use.multicast=false
-nifi.cluster.protocol.multicast.address=
-nifi.cluster.protocol.multicast.port=
-nifi.cluster.protocol.multicast.service.broadcast.delay=500 ms
-nifi.cluster.protocol.multicast.service.locator.attempts=3
-nifi.cluster.protocol.multicast.service.locator.attempts.delay=1 sec
-
-# cluster node properties (only configure for cluster nodes) #
-nifi.cluster.is.node=false
-nifi.cluster.node.address=
-nifi.cluster.node.protocol.port=
-nifi.cluster.node.protocol.threads=2
-# if multicast is not used, nifi.cluster.node.unicast.xxx must have same values as nifi.cluster.manager.xxx #
-nifi.cluster.node.unicast.manager.address=
-nifi.cluster.node.unicast.manager.protocol.port=
-nifi.cluster.node.unicast.manager.authority.provider.port=
-
-# cluster manager properties (only configure for cluster manager) #
-nifi.cluster.is.manager=false
-nifi.cluster.manager.address=
-nifi.cluster.manager.protocol.port=
-nifi.cluster.manager.authority.provider.port=
-nifi.cluster.manager.authority.provider.threads=10
-nifi.cluster.manager.node.firewall.file=
-nifi.cluster.manager.node.event.history.size=10
-nifi.cluster.manager.node.api.connection.timeout=30 sec
-nifi.cluster.manager.node.api.read.timeout=30 sec
-nifi.cluster.manager.node.api.request.threads=10
-nifi.cluster.manager.flow.retrieval.delay=5 sec
-nifi.cluster.manager.protocol.threads=10
-nifi.cluster.manager.safemode.duration=0 sec

http://git-wip-us.apache.org/repos/asf/nifi/blob/556f309d/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/test/resources/NarUnpacker/lib/dummy-one.nar
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/test/resources/NarUnpacker/lib/dummy-one.nar b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/test/resources/NarUnpacker/lib/dummy-one.nar
deleted file mode 100644
index 598b27f..0000000
Binary files a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/test/resources/NarUnpacker/lib/dummy-one.nar and /dev/null differ

http://git-wip-us.apache.org/repos/asf/nifi/blob/556f309d/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/test/resources/NarUnpacker/lib/nifi-framework-nar.nar
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/test/resources/NarUnpacker/lib/nifi-framework-nar.nar b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/test/resources/NarUnpacker/lib/nifi-framework-nar.nar
deleted file mode 100644
index d2a8b96..0000000
Binary files a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/test/resources/NarUnpacker/lib/nifi-framework-nar.nar and /dev/null differ

http://git-wip-us.apache.org/repos/asf/nifi/blob/556f309d/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/test/resources/NarUnpacker/lib2/dummy-two.nar
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/test/resources/NarUnpacker/lib2/dummy-two.nar b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/test/resources/NarUnpacker/lib2/dummy-two.nar
deleted file mode 100644
index a1021ba..0000000
Binary files a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/test/resources/NarUnpacker/lib2/dummy-two.nar and /dev/null differ

http://git-wip-us.apache.org/repos/asf/nifi/blob/556f309d/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-runtime/pom.xml
----------------------------------------------------------------------
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 44ec7ea..f97cc69 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
@@ -44,11 +44,6 @@
             <scope>provided</scope>
         </dependency>
         <dependency>
-            <groupId>org.apache.nifi</groupId>
-            <artifactId>nifi-documentation</artifactId>
-            <scope>compile</scope>
-        </dependency>
-        <dependency>
             <groupId>org.slf4j</groupId>
             <artifactId>jul-to-slf4j</artifactId>
         </dependency>

http://git-wip-us.apache.org/repos/asf/nifi/blob/556f309d/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-runtime/src/main/java/org/apache/nifi/NiFi.java
----------------------------------------------------------------------
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 32cbeba..902b088 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
@@ -17,11 +17,10 @@
 package org.apache.nifi;
 
 import org.apache.nifi.bundle.Bundle;
-import org.apache.nifi.documentation.DocGenerator;
-import org.apache.nifi.nar.ExtensionManager;
 import org.apache.nifi.nar.ExtensionMapping;
 import org.apache.nifi.nar.NarClassLoaders;
 import org.apache.nifi.nar.NarUnpacker;
+import org.apache.nifi.nar.SystemBundle;
 import org.apache.nifi.util.FileUtils;
 import org.apache.nifi.util.NiFiProperties;
 import org.slf4j.Logger;
@@ -126,7 +125,7 @@ public class NiFi {
         SLF4JBridgeHandler.removeHandlersForRootLogger();
         SLF4JBridgeHandler.install();
 
-        final Bundle systemBundle = ExtensionManager.createSystemBundle(properties);
+        final Bundle systemBundle = SystemBundle.create(properties);
 
         // expand the nars
         final ExtensionMapping extensionMapping = NarUnpacker.unpackNars(properties, systemBundle);
@@ -140,12 +139,7 @@ public class NiFi {
             throw new IllegalStateException("Unable to find the framework NAR ClassLoader.");
         }
 
-        // discover the extensions
         final Set<Bundle> narBundles = NarClassLoaders.getInstance().getBundles();
-        ExtensionManager.discoverExtensions(systemBundle, narBundles);
-        ExtensionManager.logClassLoaderMapping();
-
-        DocGenerator.generate(properties, extensionMapping);
 
         // load the server from the framework classloader
         Thread.currentThread().setContextClassLoader(frameworkClassLoader);
@@ -155,6 +149,7 @@ public class NiFi {
         final long startTime = System.nanoTime();
         nifiServer = (NiFiServer) jettyConstructor.newInstance(properties, narBundles);
         nifiServer.setExtensionMapping(extensionMapping);
+        nifiServer.setBundles(systemBundle, narBundles);
 
         if (shutdown) {
             LOGGER.info("NiFi has been shutdown via NiFi Bootstrap. Will not start Controller");

http://git-wip-us.apache.org/repos/asf/nifi/blob/556f309d/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-runtime/src/main/java/org/apache/nifi/NiFiServer.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-runtime/src/main/java/org/apache/nifi/NiFiServer.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-runtime/src/main/java/org/apache/nifi/NiFiServer.java
index 952a926..edb8f45 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-runtime/src/main/java/org/apache/nifi/NiFiServer.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-runtime/src/main/java/org/apache/nifi/NiFiServer.java
@@ -16,8 +16,11 @@
  */
 package org.apache.nifi;
 
+import org.apache.nifi.bundle.Bundle;
 import org.apache.nifi.nar.ExtensionMapping;
 
+import java.util.Set;
+
 /**
  *
  */
@@ -27,5 +30,7 @@ public interface NiFiServer {
 
     void setExtensionMapping(ExtensionMapping extensionMapping);
 
+    void setBundles(Bundle systemBundle, Set<Bundle> bundles);
+
     void stop();
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/556f309d/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-jetty/pom.xml
----------------------------------------------------------------------
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 90db844..cf75ddf 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
@@ -35,6 +35,11 @@
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-documentation</artifactId>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-nar-utils</artifactId>
             <scope>compile</scope>
         </dependency>