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:49 UTC

[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

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