You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@nifi.apache.org by ma...@apache.org on 2018/11/01 18:41:39 UTC

[1/7] nifi git commit: NIFI-5673 Set up property/assembly for new auto-load directory - Set up NarAutoLoader to watch directory for new files - Move NarAutoLoader to JettyServer since it will need access to ExtensionManager - Created NarLoader to shared

Repository: nifi
Updated Branches:
  refs/heads/master 89295e52e -> fdd8cdbb3


http://git-wip-us.apache.org/repos/asf/nifi/blob/fdd8cdbb/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/VersionsResource.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/VersionsResource.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/VersionsResource.java
index 394048c..b9cbbd0 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/VersionsResource.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/VersionsResource.java
@@ -44,7 +44,6 @@ import org.apache.nifi.registry.flow.VersionedFlowSnapshot;
 import org.apache.nifi.registry.flow.VersionedFlowSnapshotMetadata;
 import org.apache.nifi.registry.flow.VersionedFlowState;
 import org.apache.nifi.registry.flow.VersionedProcessGroup;
-import org.apache.nifi.util.BundleUtils;
 import org.apache.nifi.web.NiFiServiceFacade;
 import org.apache.nifi.web.ResourceNotFoundException;
 import org.apache.nifi.web.ResumeFlowException;
@@ -1139,7 +1138,7 @@ public class VersionsResource extends ApplicationResource {
 
         // The flow in the registry may not contain the same versions of components that we have in our flow. As a result, we need to update
         // the flow snapshot to contain compatible bundles.
-        BundleUtils.discoverCompatibleBundles(flowSnapshot.getFlowContents());
+        serviceFacade.discoverCompatibleBundles(flowSnapshot.getFlowContents());
 
         // Step 1: Determine which components will be affected by updating the version
         final Set<AffectedComponentEntity> affectedComponents = serviceFacade.getComponentsAffectedByVersionChange(groupId, flowSnapshot);
@@ -1160,7 +1159,7 @@ public class VersionsResource extends ApplicationResource {
                 authorizeProcessGroup(groupAuthorizable, authorizer, lookup, RequestAction.WRITE, true, false, true, true);
 
                 final VersionedProcessGroup groupContents = flowSnapshot.getFlowContents();
-                final Set<ConfigurableComponent> restrictedComponents = FlowRegistryUtils.getRestrictedComponents(groupContents);
+                final Set<ConfigurableComponent> restrictedComponents = FlowRegistryUtils.getRestrictedComponents(groupContents, serviceFacade);
                 restrictedComponents.forEach(restrictedComponent -> {
                     final ComponentAuthorizable restrictedComponentAuthorizable = lookup.getConfigurableComponent(restrictedComponent);
                     authorizeRestrictions(authorizer, restrictedComponentAuthorizable);
@@ -1300,7 +1299,7 @@ public class VersionsResource extends ApplicationResource {
 
         // The flow in the registry may not contain the same versions of components that we have in our flow. As a result, we need to update
         // the flow snapshot to contain compatible bundles.
-        BundleUtils.discoverCompatibleBundles(flowSnapshot.getFlowContents());
+        serviceFacade.discoverCompatibleBundles(flowSnapshot.getFlowContents());
 
         // Step 1: Determine which components will be affected by updating the version
         final Set<AffectedComponentEntity> affectedComponents = serviceFacade.getComponentsAffectedByVersionChange(groupId, flowSnapshot);
@@ -1321,7 +1320,7 @@ public class VersionsResource extends ApplicationResource {
                 authorizeProcessGroup(groupAuthorizable, authorizer, lookup, RequestAction.WRITE, true, false, true, true);
 
                 final VersionedProcessGroup groupContents = flowSnapshot.getFlowContents();
-                final Set<ConfigurableComponent> restrictedComponents = FlowRegistryUtils.getRestrictedComponents(groupContents);
+                final Set<ConfigurableComponent> restrictedComponents = FlowRegistryUtils.getRestrictedComponents(groupContents, serviceFacade);
                 restrictedComponents.forEach(restrictedComponent -> {
                     final ComponentAuthorizable restrictedComponentAuthorizable = lookup.getConfigurableComponent(restrictedComponent);
                     authorizeRestrictions(authorizer, restrictedComponentAuthorizable);

http://git-wip-us.apache.org/repos/asf/nifi/blob/fdd8cdbb/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/DtoFactory.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/dto/DtoFactory.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/DtoFactory.java
index 1393608..562d6bc 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/DtoFactory.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/DtoFactory.java
@@ -118,7 +118,7 @@ import org.apache.nifi.groups.RemoteProcessGroup;
 import org.apache.nifi.groups.RemoteProcessGroupCounts;
 import org.apache.nifi.history.History;
 import org.apache.nifi.nar.ExtensionManager;
-import org.apache.nifi.nar.NarClassLoaders;
+import org.apache.nifi.nar.NarClassLoadersHolder;
 import org.apache.nifi.processor.Processor;
 import org.apache.nifi.processor.Relationship;
 import org.apache.nifi.provenance.lineage.ComputeLineageResult;
@@ -265,6 +265,7 @@ public final class DtoFactory {
     private ControllerServiceProvider controllerServiceProvider;
     private EntityFactory entityFactory;
     private Authorizer authorizer;
+    private ExtensionManager extensionManager;
 
     public ControllerConfigurationDTO createControllerConfigurationDto(final ControllerFacade controllerFacade) {
         final ControllerConfigurationDTO dto = new ControllerConfigurationDTO();
@@ -1346,7 +1347,7 @@ public final class DtoFactory {
 
     public ReportingTaskDTO createReportingTaskDto(final ReportingTaskNode reportingTaskNode) {
         final BundleCoordinate bundleCoordinate = reportingTaskNode.getBundleCoordinate();
-        final List<Bundle> compatibleBundles = ExtensionManager.getBundles(reportingTaskNode.getCanonicalClassName()).stream().filter(bundle -> {
+        final List<Bundle> compatibleBundles = extensionManager.getBundles(reportingTaskNode.getCanonicalClassName()).stream().filter(bundle -> {
             final BundleCoordinate coordinate = bundle.getBundleDetails().getCoordinate();
             return bundleCoordinate.getGroup().equals(coordinate.getGroup()) && bundleCoordinate.getId().equals(coordinate.getId());
         }).collect(Collectors.toList());
@@ -1431,7 +1432,7 @@ public final class DtoFactory {
 
     public ControllerServiceDTO createControllerServiceDto(final ControllerServiceNode controllerServiceNode) {
         final BundleCoordinate bundleCoordinate = controllerServiceNode.getBundleCoordinate();
-        final List<Bundle> compatibleBundles = ExtensionManager.getBundles(controllerServiceNode.getCanonicalClassName()).stream().filter(bundle -> {
+        final List<Bundle> compatibleBundles = extensionManager.getBundles(controllerServiceNode.getCanonicalClassName()).stream().filter(bundle -> {
             final BundleCoordinate coordinate = bundle.getBundleDetails().getCoordinate();
             return bundleCoordinate.getGroup().equals(coordinate.getGroup()) && bundleCoordinate.getId().equals(coordinate.getId());
         }).collect(Collectors.toList());
@@ -2727,7 +2728,7 @@ public final class DtoFactory {
 
             final List<ControllerServiceApiDTO> dtos = new ArrayList<>();
             for (final Class serviceApi : serviceApis) {
-                final Bundle bundle = ExtensionManager.getBundle(serviceApi.getClassLoader());
+                final Bundle bundle = extensionManager.getBundle(serviceApi.getClassLoader());
                 final BundleCoordinate bundleCoordinate = bundle.getBundleDetails().getCoordinate();
 
                 final ControllerServiceApiDTO dto = new ControllerServiceApiDTO();
@@ -2798,7 +2799,7 @@ public final class DtoFactory {
     public Set<DocumentedTypeDTO> fromDocumentedTypes(final Set<Class> classes, final String bundleGroupFilter, final String bundleArtifactFilter, final String typeFilter) {
         final Map<Class, Bundle> classBundles = new HashMap<>();
         for (final Class cls : classes) {
-            classBundles.put(cls, ExtensionManager.getBundle(cls.getClassLoader()));
+            classBundles.put(cls, extensionManager.getBundle(cls.getClassLoader()));
         }
         return fromDocumentedTypes(classBundles, bundleGroupFilter, bundleArtifactFilter, typeFilter);
     }
@@ -2815,7 +2816,7 @@ public final class DtoFactory {
         }
 
         final BundleCoordinate bundleCoordinate = node.getBundleCoordinate();
-        final List<Bundle> compatibleBundles = ExtensionManager.getBundles(node.getCanonicalClassName()).stream().filter(bundle -> {
+        final List<Bundle> compatibleBundles = extensionManager.getBundles(node.getCanonicalClassName()).stream().filter(bundle -> {
             final BundleCoordinate coordinate = bundle.getBundleDetails().getCoordinate();
             return bundleCoordinate.getGroup().equals(coordinate.getGroup()) && bundleCoordinate.getId().equals(coordinate.getId());
         }).collect(Collectors.toList());
@@ -3219,7 +3220,7 @@ public final class DtoFactory {
         dto.setOsVersion(System.getProperty("os.version"));
         dto.setOsArchitecture(System.getProperty("os.arch"));
 
-        final Bundle frameworkBundle = NarClassLoaders.getInstance().getFrameworkBundle();
+        final Bundle frameworkBundle = NarClassLoadersHolder.getInstance().getFrameworkBundle();
         if (frameworkBundle != null) {
             final BundleDetails frameworkDetails = frameworkBundle.getBundleDetails();
 
@@ -3331,7 +3332,7 @@ public final class DtoFactory {
 
 
     private ClassLoaderDiagnosticsDTO createClassLoaderDiagnosticsDto(final ControllerServiceNode serviceNode) {
-        ClassLoader componentClassLoader = ExtensionManager.getInstanceClassLoader(serviceNode.getIdentifier());
+        ClassLoader componentClassLoader = extensionManager.getInstanceClassLoader(serviceNode.getIdentifier());
         if (componentClassLoader == null) {
             componentClassLoader = serviceNode.getControllerServiceImplementation().getClass().getClassLoader();
         }
@@ -3341,7 +3342,7 @@ public final class DtoFactory {
 
 
     private ClassLoaderDiagnosticsDTO createClassLoaderDiagnosticsDto(final ProcessorNode procNode) {
-        ClassLoader componentClassLoader = ExtensionManager.getInstanceClassLoader(procNode.getIdentifier());
+        ClassLoader componentClassLoader = extensionManager.getInstanceClassLoader(procNode.getIdentifier());
         if (componentClassLoader == null) {
             componentClassLoader = procNode.getProcessor().getClass().getClassLoader();
         }
@@ -3352,7 +3353,7 @@ public final class DtoFactory {
     private ClassLoaderDiagnosticsDTO createClassLoaderDiagnosticsDto(final ClassLoader classLoader) {
         final ClassLoaderDiagnosticsDTO dto = new ClassLoaderDiagnosticsDTO();
 
-        final Bundle bundle = ExtensionManager.getBundle(classLoader);
+        final Bundle bundle = extensionManager.getBundle(classLoader);
         if (bundle != null) {
             dto.setBundle(createBundleDto(bundle.getBundleDetails().getCoordinate()));
         }
@@ -3469,7 +3470,7 @@ public final class DtoFactory {
         final SystemDiagnostics systemDiagnostics = flowController.getSystemDiagnostics();
 
         // flow-related information
-        final Set<BundleDTO> bundlesLoaded = ExtensionManager.getAllBundles().stream()
+        final Set<BundleDTO> bundlesLoaded = extensionManager.getAllBundles().stream()
             .map(bundle -> bundle.getBundleDetails().getCoordinate())
             .sorted((a, b) -> a.getCoordinate().compareTo(b.getCoordinate()))
             .map(this::createBundleDto)
@@ -3719,7 +3720,7 @@ public final class DtoFactory {
         // set the identifies controller service is applicable
         if (propertyDescriptor.getControllerServiceDefinition() != null) {
             final Class serviceClass = propertyDescriptor.getControllerServiceDefinition();
-            final Bundle serviceBundle = ExtensionManager.getBundle(serviceClass.getClassLoader());
+            final Bundle serviceBundle = extensionManager.getBundle(serviceClass.getClassLoader());
 
             dto.setIdentifiesControllerService(serviceClass.getName());
             dto.setIdentifiesControllerServiceBundle(createBundleDto(serviceBundle.getBundleDetails().getCoordinate()));
@@ -4327,4 +4328,8 @@ public final class DtoFactory {
     public void setBulletinRepository(BulletinRepository bulletinRepository) {
         this.bulletinRepository = bulletinRepository;
     }
+
+    public void setExtensionManager(ExtensionManager extensionManager) {
+        this.extensionManager = extensionManager;
+    }
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/fdd8cdbb/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 18a47d2..88b13a0 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
@@ -174,6 +174,10 @@ public class ControllerFacade implements Authorizable {
         return flowController;
     }
 
+    public ExtensionManager getExtensionManager() {
+        return flowController.getExtensionManager();
+    }
+
     /**
      * Sets the name of this controller.
      *
@@ -211,7 +215,9 @@ public class ControllerFacade implements Authorizable {
      * @throws IllegalStateException if no temporary component exists for the given type and bundle
      */
     public ConfigurableComponent getTemporaryComponent(final String type, final BundleDTO bundle) {
-        final ConfigurableComponent configurableComponent = ExtensionManager.getTempComponent(type, BundleUtils.getBundle(type, bundle));
+        final ExtensionManager extensionManager = getExtensionManager();
+        final BundleCoordinate bundleCoordinate = BundleUtils.getBundle(extensionManager, type, bundle);
+        final ConfigurableComponent configurableComponent = extensionManager.getTempComponent(type, bundleCoordinate);
 
         if (configurableComponent == null) {
             throw new IllegalStateException("Unable to obtain temporary component for " + type);
@@ -451,7 +457,7 @@ public class ControllerFacade implements Authorizable {
      * @return types
      */
     public Set<DocumentedTypeDTO> getFlowFileProcessorTypes(final String bundleGroupFilter, final String bundleArtifactFilter, final String typeFilter) {
-        return dtoFactory.fromDocumentedTypes(ExtensionManager.getExtensions(Processor.class), bundleGroupFilter, bundleArtifactFilter, typeFilter);
+        return dtoFactory.fromDocumentedTypes(getExtensionManager().getExtensions(Processor.class), bundleGroupFilter, bundleArtifactFilter, typeFilter);
     }
 
     /**
@@ -460,7 +466,7 @@ public class ControllerFacade implements Authorizable {
      * @return the FlowFileComparator types that this controller supports
      */
     public Set<DocumentedTypeDTO> getFlowFileComparatorTypes() {
-        return dtoFactory.fromDocumentedTypes(ExtensionManager.getExtensions(FlowFilePrioritizer.class), null, null, null);
+        return dtoFactory.fromDocumentedTypes(getExtensionManager().getExtensions(FlowFilePrioritizer.class), null, null, null);
     }
 
     /**
@@ -496,12 +502,12 @@ public class ControllerFacade implements Authorizable {
     public Set<DocumentedTypeDTO> getControllerServiceTypes(final String serviceType, final String serviceBundleGroup, final String serviceBundleArtifact, final String serviceBundleVersion,
                                                             final String bundleGroupFilter, final String bundleArtifactFilter, final String typeFilter) {
 
-        final Set<Class> serviceImplementations = ExtensionManager.getExtensions(ControllerService.class);
+        final Set<Class> serviceImplementations = getExtensionManager().getExtensions(ControllerService.class);
 
         // identify the controller services that implement the specified serviceType if applicable
         if (serviceType != null) {
             final BundleCoordinate bundleCoordinate = new BundleCoordinate(serviceBundleGroup, serviceBundleArtifact, serviceBundleVersion);
-            final Bundle csBundle = ExtensionManager.getBundle(bundleCoordinate);
+            final Bundle csBundle = getExtensionManager().getBundle(bundleCoordinate);
             if (csBundle == null) {
                 throw new IllegalStateException("Unable to find bundle for coordinate " + bundleCoordinate.getCoordinate());
             }
@@ -521,7 +527,7 @@ public class ControllerFacade implements Authorizable {
             // check each type and remove those that aren't in the specified ancestry
             for (final Class csClass : serviceImplementations) {
                 if (implementsServiceType(serviceClass, csClass)) {
-                    matchingServiceImplementations.put(csClass, ExtensionManager.getBundle(csClass.getClassLoader()));
+                    matchingServiceImplementations.put(csClass, getExtensionManager().getBundle(csClass.getClassLoader()));
                 }
             }
 
@@ -540,7 +546,7 @@ public class ControllerFacade implements Authorizable {
      * @return the ReportingTask types that this controller supports
      */
     public Set<DocumentedTypeDTO> getReportingTaskTypes(final String bundleGroupFilter, final String bundleArtifactFilter, final String typeFilter) {
-        return dtoFactory.fromDocumentedTypes(ExtensionManager.getExtensions(ReportingTask.class), bundleGroupFilter, bundleArtifactFilter, typeFilter);
+        return dtoFactory.fromDocumentedTypes(getExtensionManager().getExtensions(ReportingTask.class), bundleGroupFilter, bundleArtifactFilter, typeFilter);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/nifi/blob/fdd8cdbb/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerSearchService.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/ControllerSearchService.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerSearchService.java
index ae84db0..f1a90c6 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerSearchService.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerSearchService.java
@@ -292,7 +292,7 @@ public class ControllerSearchService {
             final SearchContext context = new StandardSearchContext(searchStr, procNode, flowController, variableRegistry);
 
             // search the processor using the appropriate thread context classloader
-            try (final NarCloseable x = NarCloseable.withComponentNarLoader(processor.getClass(), processor.getIdentifier())) {
+            try (final NarCloseable x = NarCloseable.withComponentNarLoader(flowController.getExtensionManager(), processor.getClass(), processor.getIdentifier())) {
                 final Collection<SearchResult> searchResults = searchable.search(context);
                 if (CollectionUtils.isNotEmpty(searchResults)) {
                     for (final SearchResult searchResult : searchResults) {

http://git-wip-us.apache.org/repos/asf/nifi/blob/fdd8cdbb/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/ComponentDAO.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/ComponentDAO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/ComponentDAO.java
index 42f4fd5..aaec17a 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/ComponentDAO.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/ComponentDAO.java
@@ -73,8 +73,8 @@ public abstract class ComponentDAO {
         return group;
     }
 
-    protected void verifyCreate(final String type, final BundleDTO bundle) {
-        final List<Bundle> bundles = ExtensionManager.getBundles(type);
+    protected void verifyCreate(final ExtensionManager extensionManager, final String type, final BundleDTO bundle) {
+        final List<Bundle> bundles = extensionManager.getBundles(type);
 
         if (bundle != null) {
             final BundleCoordinate coordinate = new BundleCoordinate(bundle.getGroup(), bundle.getArtifact(), bundle.getVersion());

http://git-wip-us.apache.org/repos/asf/nifi/blob/fdd8cdbb/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 953e03a..2995bae 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
@@ -67,7 +67,7 @@ public class StandardControllerServiceDAO extends ComponentDAO implements Contro
 
     @Override
     public void verifyCreate(final ControllerServiceDTO controllerServiceDTO) {
-        verifyCreate(controllerServiceDTO.getType(), controllerServiceDTO.getBundle());
+        verifyCreate(serviceProvider.getExtensionManager(), controllerServiceDTO.getType(), controllerServiceDTO.getBundle());
     }
 
     @Override
@@ -79,9 +79,10 @@ public class StandardControllerServiceDAO extends ComponentDAO implements Contro
 
         try {
             // create the controller service
+            final ExtensionManager extensionManager = serviceProvider.getExtensionManager();
+            final BundleCoordinate bundleCoordinate = BundleUtils.getBundle(extensionManager, controllerServiceDTO.getType(), controllerServiceDTO.getBundle());
             final ControllerServiceNode controllerService = serviceProvider.createControllerService(
-                    controllerServiceDTO.getType(), controllerServiceDTO.getId(), BundleUtils.getBundle(controllerServiceDTO.getType(),
-                            controllerServiceDTO.getBundle()), Collections.emptySet(), true);
+                    controllerServiceDTO.getType(), controllerServiceDTO.getId(), bundleCoordinate, Collections.emptySet(), true);
 
             // ensure we can perform the update
             verifyUpdate(controllerService, controllerServiceDTO);
@@ -196,12 +197,13 @@ public class StandardControllerServiceDAO extends ComponentDAO implements Contro
     private void updateBundle(final ControllerServiceNode controllerService, final ControllerServiceDTO controllerServiceDTO) {
         final BundleDTO bundleDTO = controllerServiceDTO.getBundle();
         if (bundleDTO != null) {
-            final BundleCoordinate incomingCoordinate = BundleUtils.getBundle(controllerService.getCanonicalClassName(), bundleDTO);
+            final ExtensionManager extensionManager = serviceProvider.getExtensionManager();
+            final BundleCoordinate incomingCoordinate = BundleUtils.getBundle(extensionManager, controllerService.getCanonicalClassName(), bundleDTO);
             final BundleCoordinate existingCoordinate = controllerService.getBundleCoordinate();
             if (!existingCoordinate.getCoordinate().equals(incomingCoordinate.getCoordinate())) {
                 try {
                     // we need to use the property descriptors from the temp component here in case we are changing from a ghost component to a real component
-                    final ConfigurableComponent tempComponent = ExtensionManager.getTempComponent(controllerService.getCanonicalClassName(), incomingCoordinate);
+                    final ConfigurableComponent tempComponent = extensionManager.getTempComponent(controllerService.getCanonicalClassName(), incomingCoordinate);
                     final Set<URL> additionalUrls = controllerService.getAdditionalClasspathResources(tempComponent.getPropertyDescriptors());
                     flowController.reload(controllerService, controllerService.getCanonicalClassName(), incomingCoordinate, additionalUrls);
                 } catch (ControllerServiceInstantiationException e) {
@@ -317,7 +319,7 @@ public class StandardControllerServiceDAO extends ComponentDAO implements Contro
         final BundleDTO bundleDTO = controllerServiceDTO.getBundle();
         if (bundleDTO != null) {
             // ensures all nodes in a cluster have the bundle, throws exception if bundle not found for the given type
-            final BundleCoordinate bundleCoordinate = BundleUtils.getBundle(controllerService.getCanonicalClassName(), bundleDTO);
+            final BundleCoordinate bundleCoordinate = BundleUtils.getBundle(serviceProvider.getExtensionManager(), controllerService.getCanonicalClassName(), bundleDTO);
             // ensure we are only changing to a bundle with the same group and id, but different version
             controllerService.verifyCanUpdateBundle(bundleCoordinate);
         }

http://git-wip-us.apache.org/repos/asf/nifi/blob/fdd8cdbb/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardProcessGroupDAO.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/StandardProcessGroupDAO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardProcessGroupDAO.java
index 06aab80..47e9855 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardProcessGroupDAO.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardProcessGroupDAO.java
@@ -328,7 +328,7 @@ public class StandardProcessGroupDAO extends ComponentDAO implements ProcessGrou
         final FlowRegistry flowRegistry = flowController.getFlowRegistryClient().getFlowRegistry(registryId);
         final String registryName = flowRegistry == null ? registryId : flowRegistry.getName();
 
-        final NiFiRegistryFlowMapper mapper = new NiFiRegistryFlowMapper();
+        final NiFiRegistryFlowMapper mapper = new NiFiRegistryFlowMapper(flowController.getExtensionManager());
         final VersionedProcessGroup flowSnapshot = mapper.mapProcessGroup(group, flowController, flowController.getFlowRegistryClient(), false);
 
         final StandardVersionControlInformation vci = StandardVersionControlInformation.Builder.fromDto(versionControlInformation)

http://git-wip-us.apache.org/repos/asf/nifi/blob/fdd8cdbb/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 5a1f3bb..1de4bff 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
@@ -85,7 +85,7 @@ public class StandardProcessorDAO extends ComponentDAO implements ProcessorDAO {
 
     @Override
     public void verifyCreate(final ProcessorDTO processorDTO) {
-        verifyCreate(processorDTO.getType(), processorDTO.getBundle());
+        verifyCreate(flowController.getExtensionManager(), processorDTO.getType(), processorDTO.getBundle());
     }
 
     @Override
@@ -104,7 +104,8 @@ public class StandardProcessorDAO extends ComponentDAO implements ProcessorDAO {
 
         try {
             // attempt to create the processor
-            ProcessorNode processor = flowController.createProcessor(processorDTO.getType(), processorDTO.getId(), BundleUtils.getBundle(processorDTO.getType(), processorDTO.getBundle()));
+            final BundleCoordinate bundleCoordinate = BundleUtils.getBundle(flowController.getExtensionManager(), processorDTO.getType(), processorDTO.getBundle());
+            ProcessorNode processor = flowController.createProcessor(processorDTO.getType(), processorDTO.getId(), bundleCoordinate);
 
             // ensure we can perform the update before we add the processor to the flow
             verifyUpdate(processor, processorDTO);
@@ -383,7 +384,7 @@ public class StandardProcessorDAO extends ComponentDAO implements ProcessorDAO {
         final BundleDTO bundleDTO = processorDTO.getBundle();
         if (bundleDTO != null) {
             // ensures all nodes in a cluster have the bundle, throws exception if bundle not found for the given type
-            final BundleCoordinate bundleCoordinate = BundleUtils.getBundle(processor.getCanonicalClassName(), bundleDTO);
+            final BundleCoordinate bundleCoordinate = BundleUtils.getBundle(flowController.getExtensionManager(), processor.getCanonicalClassName(), bundleDTO);
             // ensure we are only changing to a bundle with the same group and id, but different version
             processor.verifyCanUpdateBundle(bundleCoordinate);
         }
@@ -479,12 +480,13 @@ public class StandardProcessorDAO extends ComponentDAO implements ProcessorDAO {
     private void updateBundle(ProcessorNode processor, ProcessorDTO processorDTO) {
         final BundleDTO bundleDTO = processorDTO.getBundle();
         if (bundleDTO != null) {
-            final BundleCoordinate incomingCoordinate = BundleUtils.getBundle(processor.getCanonicalClassName(), bundleDTO);
+            final ExtensionManager extensionManager = flowController.getExtensionManager();
+            final BundleCoordinate incomingCoordinate = BundleUtils.getBundle(extensionManager, processor.getCanonicalClassName(), bundleDTO);
             final BundleCoordinate existingCoordinate = processor.getBundleCoordinate();
             if (!existingCoordinate.getCoordinate().equals(incomingCoordinate.getCoordinate())) {
                 try {
                     // we need to use the property descriptors from the temp component here in case we are changing from a ghost component to a real component
-                    final ConfigurableComponent tempComponent = ExtensionManager.getTempComponent(processor.getCanonicalClassName(), incomingCoordinate);
+                    final ConfigurableComponent tempComponent = extensionManager.getTempComponent(processor.getCanonicalClassName(), incomingCoordinate);
                     final Set<URL> additionalUrls = processor.getAdditionalClasspathResources(tempComponent.getPropertyDescriptors());
                     flowController.reload(processor, processor.getCanonicalClassName(), incomingCoordinate, additionalUrls);
                 } catch (ProcessorInstantiationException e) {

http://git-wip-us.apache.org/repos/asf/nifi/blob/fdd8cdbb/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 5210a08..939a2d5 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
@@ -69,7 +69,7 @@ public class StandardReportingTaskDAO extends ComponentDAO implements ReportingT
 
     @Override
     public void verifyCreate(final ReportingTaskDTO reportingTaskDTO) {
-        verifyCreate(reportingTaskDTO.getType(), reportingTaskDTO.getBundle());
+        verifyCreate(reportingTaskProvider.getExtensionManager(), reportingTaskDTO.getType(), reportingTaskDTO.getBundle());
     }
 
     @Override
@@ -81,8 +81,10 @@ public class StandardReportingTaskDAO extends ComponentDAO implements ReportingT
 
         try {
             // create the reporting task
+            final ExtensionManager extensionManager = reportingTaskProvider.getExtensionManager();
+            final BundleCoordinate bundleCoordinate = BundleUtils.getBundle(extensionManager, reportingTaskDTO.getType(), reportingTaskDTO.getBundle());
             final ReportingTaskNode reportingTask = reportingTaskProvider.createReportingTask(
-                    reportingTaskDTO.getType(), reportingTaskDTO.getId(), BundleUtils.getBundle(reportingTaskDTO.getType(), reportingTaskDTO.getBundle()), true);
+                    reportingTaskDTO.getType(), reportingTaskDTO.getId(), bundleCoordinate, true);
 
             // ensure we can perform the update
             verifyUpdate(reportingTask, reportingTaskDTO);
@@ -171,12 +173,13 @@ public class StandardReportingTaskDAO extends ComponentDAO implements ReportingT
     private void updateBundle(ReportingTaskNode reportingTask, ReportingTaskDTO reportingTaskDTO) {
         final BundleDTO bundleDTO = reportingTaskDTO.getBundle();
         if (bundleDTO != null) {
-            final BundleCoordinate incomingCoordinate = BundleUtils.getBundle(reportingTask.getCanonicalClassName(), bundleDTO);
+            final ExtensionManager extensionManager = reportingTaskProvider.getExtensionManager();
+            final BundleCoordinate incomingCoordinate = BundleUtils.getBundle(extensionManager, reportingTask.getCanonicalClassName(), bundleDTO);
             final BundleCoordinate existingCoordinate = reportingTask.getBundleCoordinate();
             if (!existingCoordinate.getCoordinate().equals(incomingCoordinate.getCoordinate())) {
                 try {
                     // we need to use the property descriptors from the temp component here in case we are changing from a ghost component to a real component
-                    final ConfigurableComponent tempComponent = ExtensionManager.getTempComponent(reportingTask.getCanonicalClassName(), incomingCoordinate);
+                    final ConfigurableComponent tempComponent = extensionManager.getTempComponent(reportingTask.getCanonicalClassName(), incomingCoordinate);
                     final Set<URL> additionalUrls = reportingTask.getAdditionalClasspathResources(tempComponent.getPropertyDescriptors());
                     reloadComponent.reload(reportingTask, reportingTask.getCanonicalClassName(), incomingCoordinate, additionalUrls);
                 } catch (ReportingTaskInstantiationException e) {
@@ -295,7 +298,8 @@ public class StandardReportingTaskDAO extends ComponentDAO implements ReportingT
         final BundleDTO bundleDTO = reportingTaskDTO.getBundle();
         if (bundleDTO != null) {
             // ensures all nodes in a cluster have the bundle, throws exception if bundle not found for the given type
-            final BundleCoordinate bundleCoordinate = BundleUtils.getBundle(reportingTask.getCanonicalClassName(), bundleDTO);
+            final BundleCoordinate bundleCoordinate = BundleUtils.getBundle(
+                    reportingTaskProvider.getExtensionManager(), reportingTask.getCanonicalClassName(), bundleDTO);
             // ensure we are only changing to a bundle with the same group and id, but different version
             reportingTask.verifyCanUpdateBundle(bundleCoordinate);
         }

http://git-wip-us.apache.org/repos/asf/nifi/blob/fdd8cdbb/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 8c1e2ad..7802229 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
@@ -52,6 +52,7 @@
         <property name="entityFactory" ref="entityFactory"/>
         <property name="authorizer" ref="authorizer"/>
         <property name="bulletinRepository" ref="bulletinRepository"/>
+        <property name="extensionManager" ref="extensionManager" />
     </bean>
 
     <!-- snippet utils -->

http://git-wip-us.apache.org/repos/asf/nifi/blob/fdd8cdbb/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/authorization/StandardAuthorizableLookupTest.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/authorization/StandardAuthorizableLookupTest.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/authorization/StandardAuthorizableLookupTest.java
index 5935aa2..7fabc54 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/authorization/StandardAuthorizableLookupTest.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/authorization/StandardAuthorizableLookupTest.java
@@ -23,6 +23,9 @@ import org.apache.nifi.authorization.resource.DataTransferAuthorizable;
 import org.apache.nifi.authorization.resource.OperationAuthorizable;
 import org.apache.nifi.authorization.resource.ProvenanceDataAuthorizable;
 import org.apache.nifi.controller.ProcessorNode;
+import org.apache.nifi.nar.ExtensionDiscoveringManager;
+import org.apache.nifi.nar.ExtensionManager;
+import org.apache.nifi.web.controller.ControllerFacade;
 import org.apache.nifi.web.dao.ProcessorDAO;
 import org.junit.Test;
 
@@ -35,6 +38,10 @@ public class StandardAuthorizableLookupTest {
 
     @Test
     public void testGetAuthorizableFromResource() {
+        final ExtensionManager extensionManager = mock(ExtensionDiscoveringManager.class);
+        final ControllerFacade controllerFacade = mock(ControllerFacade.class);
+        when(controllerFacade.getExtensionManager()).thenReturn(extensionManager);
+
         final ProcessorDAO processorDAO = mock(ProcessorDAO.class);
         final ProcessorNode processorNode = mock(ProcessorNode.class);
 
@@ -42,6 +49,7 @@ public class StandardAuthorizableLookupTest {
 
         final StandardAuthorizableLookup lookup = new StandardAuthorizableLookup();
         lookup.setProcessorDAO(processorDAO);
+        lookup.setControllerFacade(controllerFacade);
 
         Authorizable authorizable = lookup.getAuthorizableFromResource("/processors/id");
         assertTrue(authorizable instanceof ProcessorNode);

http://git-wip-us.apache.org/repos/asf/nifi/blob/fdd8cdbb/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 279da21..2571866 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
@@ -21,13 +21,18 @@ import org.apache.nifi.integration.NiFiWebApiTest;
 import org.apache.nifi.integration.util.NiFiTestAuthorizer;
 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.ExtensionDiscoveringManager;
+import org.apache.nifi.nar.ExtensionManagerHolder;
+import org.apache.nifi.nar.NarClassLoadersHolder;
+import org.apache.nifi.nar.NarUnpacker;
+import org.apache.nifi.nar.StandardExtensionDiscoveringManager;
 import org.apache.nifi.nar.SystemBundle;
 import org.apache.nifi.util.NiFiProperties;
 
 import javax.ws.rs.core.Response;
 import java.io.File;
+import java.nio.file.Files;
+import java.nio.file.StandardCopyOption;
 
 import static org.junit.Assert.assertEquals;
 
@@ -65,10 +70,23 @@ public class AccessControlHelper {
         NiFiProperties props = NiFiProperties.createBasicNiFiProperties(null, null);
         flowXmlPath = props.getProperty(NiFiProperties.FLOW_CONFIGURATION_FILE);
 
-        // load extensions
+        final File libTargetDir = new File("target/test-classes/access-control/lib");
+        libTargetDir.mkdirs();
+
+        final File libSourceDir = new File("src/test/resources/lib");
+        for (final File libFile : libSourceDir.listFiles()) {
+            final File libDestFile = new File(libTargetDir, libFile.getName());
+            Files.copy(libFile.toPath(), libDestFile.toPath(), StandardCopyOption.REPLACE_EXISTING);
+        }
+
         final Bundle systemBundle = SystemBundle.create(props);
-        NarClassLoaders.getInstance().init(props.getFrameworkWorkingDirectory(), props.getExtensionsWorkingDirectory());
-        ExtensionManager.discoverExtensions(systemBundle, NarClassLoaders.getInstance().getBundles());
+        NarUnpacker.unpackNars(props, systemBundle);
+        NarClassLoadersHolder.getInstance().init(props.getFrameworkWorkingDirectory(), props.getExtensionsWorkingDirectory());
+
+        // load extensions
+        final ExtensionDiscoveringManager extensionManager = new StandardExtensionDiscoveringManager();
+        extensionManager.discoverExtensions(systemBundle, NarClassLoadersHolder.getInstance().getBundles());
+        ExtensionManagerHolder.init(extensionManager);
 
         // start the server
         server = new NiFiTestServer("src/main/webapp", CONTEXT_PATH, props);

http://git-wip-us.apache.org/repos/asf/nifi/blob/fdd8cdbb/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 aa8c154..9b6caa8 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
@@ -21,8 +21,11 @@ import org.apache.nifi.bundle.Bundle;
 import org.apache.nifi.integration.util.NiFiTestServer;
 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.ExtensionDiscoveringManager;
+import org.apache.nifi.nar.ExtensionManagerHolder;
+import org.apache.nifi.nar.NarClassLoadersHolder;
+import org.apache.nifi.nar.NarUnpacker;
+import org.apache.nifi.nar.StandardExtensionDiscoveringManager;
 import org.apache.nifi.nar.SystemBundle;
 import org.apache.nifi.security.util.SslContextFactory;
 import org.apache.nifi.util.NiFiProperties;
@@ -43,6 +46,8 @@ import javax.net.ssl.SSLContext;
 import javax.ws.rs.client.Client;
 import javax.ws.rs.core.Response;
 import java.io.File;
+import java.nio.file.Files;
+import java.nio.file.StandardCopyOption;
 import java.util.HashMap;
 import java.util.Map;
 
@@ -71,10 +76,23 @@ public class ITAccessTokenEndpoint {
         // delete the database directory to avoid issues with re-registration in testRequestAccessUsingToken
         FileUtils.deleteDirectory(props.getDatabaseRepositoryPath().toFile());
 
-        // load extensions
+        final File libTargetDir = new File("target/test-classes/access-control/lib");
+        libTargetDir.mkdirs();
+
+        final File libSourceDir = new File("src/test/resources/lib");
+        for (final File libFile : libSourceDir.listFiles()) {
+            final File libDestFile = new File(libTargetDir, libFile.getName());
+            Files.copy(libFile.toPath(), libDestFile.toPath(), StandardCopyOption.REPLACE_EXISTING);
+        }
+
         final Bundle systemBundle = SystemBundle.create(props);
-        NarClassLoaders.getInstance().init(props.getFrameworkWorkingDirectory(), props.getExtensionsWorkingDirectory());
-        ExtensionManager.discoverExtensions(systemBundle, NarClassLoaders.getInstance().getBundles());
+        NarUnpacker.unpackNars(props, systemBundle);
+        NarClassLoadersHolder.getInstance().init(props.getFrameworkWorkingDirectory(), props.getExtensionsWorkingDirectory());
+
+        // load extensions
+        final ExtensionDiscoveringManager extensionManager = new StandardExtensionDiscoveringManager();
+        extensionManager.discoverExtensions(systemBundle, NarClassLoadersHolder.getInstance().getBundles());
+        ExtensionManagerHolder.init(extensionManager);
 
         // start the server
         SERVER = new NiFiTestServer("src/main/webapp", CONTEXT_PATH, props);

http://git-wip-us.apache.org/repos/asf/nifi/blob/fdd8cdbb/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/resources/lib/nifi-framework-nar.nar
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/resources/lib/nifi-framework-nar.nar b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/resources/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-web/nifi-web-api/src/test/resources/lib/nifi-framework-nar.nar differ

http://git-wip-us.apache.org/repos/asf/nifi/blob/fdd8cdbb/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/resources/lib/nifi-jetty-bundle.nar
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/resources/lib/nifi-jetty-bundle.nar b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/resources/lib/nifi-jetty-bundle.nar
new file mode 100644
index 0000000..6fb2c5f
Binary files /dev/null and b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/resources/lib/nifi-jetty-bundle.nar differ

http://git-wip-us.apache.org/repos/asf/nifi/blob/fdd8cdbb/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/spring/LoginIdentityProviderFactoryBean.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/spring/LoginIdentityProviderFactoryBean.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/spring/LoginIdentityProviderFactoryBean.java
index 53a4ecf..3e5fcd1 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/spring/LoginIdentityProviderFactoryBean.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/spring/LoginIdentityProviderFactoryBean.java
@@ -88,6 +88,7 @@ public class LoginIdentityProviderFactoryBean implements FactoryBean, Disposable
     }
 
     private NiFiProperties properties;
+    private ExtensionManager extensionManager;
     private LoginIdentityProvider loginIdentityProvider;
     private final Map<String, LoginIdentityProvider> loginIdentityProviders = new HashMap<>();
 
@@ -156,7 +157,7 @@ public class LoginIdentityProviderFactoryBean implements FactoryBean, Disposable
 
     private LoginIdentityProvider createLoginIdentityProvider(final String identifier, final String loginIdentityProviderClassName) throws Exception {
         // get the classloader for the specified login identity provider
-        final List<Bundle> loginIdentityProviderBundles = ExtensionManager.getBundles(loginIdentityProviderClassName);
+        final List<Bundle> loginIdentityProviderBundles = extensionManager.getBundles(loginIdentityProviderClassName);
 
         if (loginIdentityProviderBundles.size() == 0) {
             throw new Exception(String.format("The specified login identity provider class '%s' is not known to this nifi.", loginIdentityProviderClassName));
@@ -358,4 +359,9 @@ public class LoginIdentityProviderFactoryBean implements FactoryBean, Disposable
     public void setProperties(NiFiProperties properties) {
         this.properties = properties;
     }
+
+    public void setExtensionManager(ExtensionManager extensionManager) {
+        this.extensionManager = extensionManager;
+    }
+
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/fdd8cdbb/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/resources/nifi-web-security-context.xml
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/resources/nifi-web-security-context.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/resources/nifi-web-security-context.xml
index 0c1009a..32c20c8 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/resources/nifi-web-security-context.xml
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/resources/nifi-web-security-context.xml
@@ -88,6 +88,7 @@
     <!-- login identity provider -->
     <bean id="loginIdentityProvider" class="org.apache.nifi.web.security.spring.LoginIdentityProviderFactoryBean">
         <property name="properties" ref="nifiProperties"/>
+        <property name="extensionManager" ref="extensionManager" />
     </bean>
 
     <!-- oidc -->

http://git-wip-us.apache.org/repos/asf/nifi/blob/fdd8cdbb/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 dda0c49..4383ace 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/pom.xml
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/pom.xml
@@ -35,7 +35,8 @@
         <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-nar-loading-utils</module>
+	<module>nifi-user-actions</module>
         <module>nifi-framework-authorization</module>
         <module>nifi-file-authorizer</module>
         <module>nifi-administration</module>

http://git-wip-us.apache.org/repos/asf/nifi/blob/fdd8cdbb/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 3be5f0f..78044d9 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/pom.xml
+++ b/nifi-nar-bundles/nifi-framework-bundle/pom.xml
@@ -128,6 +128,11 @@
             </dependency>
             <dependency>
                 <groupId>org.apache.nifi</groupId>
+                <artifactId>nifi-framework-nar-loading-utils</artifactId>
+                <version>1.9.0-SNAPSHOT</version>
+            </dependency>
+            <dependency>
+                <groupId>org.apache.nifi</groupId>
                 <artifactId>nifi-site-to-site</artifactId>
                 <version>1.9.0-SNAPSHOT</version>
             </dependency>

http://git-wip-us.apache.org/repos/asf/nifi/blob/fdd8cdbb/nifi-nar-bundles/nifi-jetty-bundle/pom.xml
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-jetty-bundle/pom.xml b/nifi-nar-bundles/nifi-jetty-bundle/pom.xml
index e012ee3..5b5626a 100644
--- a/nifi-nar-bundles/nifi-jetty-bundle/pom.xml
+++ b/nifi-nar-bundles/nifi-jetty-bundle/pom.xml
@@ -44,6 +44,11 @@
         </dependency>
         <dependency>
             <groupId>org.eclipse.jetty</groupId>
+            <artifactId>jetty-deploy</artifactId>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.eclipse.jetty</groupId>
             <artifactId>jetty-servlets</artifactId>
             <scope>compile</scope>
         </dependency>

http://git-wip-us.apache.org/repos/asf/nifi/blob/fdd8cdbb/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 66cb040..cecfed2 100644
--- a/pom.xml
+++ b/pom.xml
@@ -264,6 +264,12 @@
             </dependency>
             <dependency>
                 <groupId>org.eclipse.jetty</groupId>
+                <artifactId>jetty-deploy</artifactId>
+                <version>${jetty.version}</version>
+                <scope>provided</scope>
+            </dependency>
+            <dependency>
+                <groupId>org.eclipse.jetty</groupId>
                 <artifactId>jetty-servlets</artifactId>
                 <version>${jetty.version}</version>
                 <scope>provided</scope>


[4/7] nifi git commit: NIFI-5673 Set up property/assembly for new auto-load directory - Set up NarAutoLoader to watch directory for new files - Move NarAutoLoader to JettyServer since it will need access to ExtensionManager - Created NarLoader to shared

Posted by ma...@apache.org.
http://git-wip-us.apache.org/repos/asf/nifi/blob/fdd8cdbb/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/scheduling/StandardProcessSchedulerIT.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/scheduling/StandardProcessSchedulerIT.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/scheduling/StandardProcessSchedulerIT.java
index f8a4b43..fac0272 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/scheduling/StandardProcessSchedulerIT.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/scheduling/StandardProcessSchedulerIT.java
@@ -17,12 +17,6 @@
 
 package org.apache.nifi.controller.scheduling;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
-import java.util.Collections;
-
 import org.apache.nifi.bundle.Bundle;
 import org.apache.nifi.components.state.StateManagerProvider;
 import org.apache.nifi.controller.FlowController;
@@ -30,7 +24,8 @@ import org.apache.nifi.controller.service.ControllerServiceNode;
 import org.apache.nifi.controller.service.ControllerServiceState;
 import org.apache.nifi.controller.service.StandardControllerServiceProvider;
 import org.apache.nifi.engine.FlowEngine;
-import org.apache.nifi.nar.ExtensionManager;
+import org.apache.nifi.nar.ExtensionDiscoveringManager;
+import org.apache.nifi.nar.StandardExtensionDiscoveringManager;
 import org.apache.nifi.nar.SystemBundle;
 import org.apache.nifi.registry.VariableRegistry;
 import org.apache.nifi.reporting.InitializationException;
@@ -40,12 +35,19 @@ import org.junit.Before;
 import org.junit.Test;
 import org.mockito.Mockito;
 
+import java.util.Collections;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
 public class StandardProcessSchedulerIT {
     private final StateManagerProvider stateMgrProvider = Mockito.mock(StateManagerProvider.class);
     private VariableRegistry variableRegistry = VariableRegistry.ENVIRONMENT_SYSTEM_REGISTRY;
     private FlowController controller;
     private NiFiProperties nifiProperties;
     private Bundle systemBundle;
+    private ExtensionDiscoveringManager extensionManager;
     private volatile String propsFile = TestStandardProcessScheduler.class.getResource("/standardprocessschedulertest.nifi.properties").getFile();
 
     @Before
@@ -54,9 +56,11 @@ public class StandardProcessSchedulerIT {
 
         // load the system bundle
         systemBundle = SystemBundle.create(nifiProperties);
-        ExtensionManager.discoverExtensions(systemBundle, Collections.emptySet());
+        extensionManager = new StandardExtensionDiscoveringManager();
+        extensionManager.discoverExtensions(systemBundle, Collections.emptySet());
 
         controller = Mockito.mock(FlowController.class);
+        Mockito.when(controller.getExtensionManager()).thenReturn(extensionManager);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/nifi/blob/fdd8cdbb/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 3c382cf..92e6e5d 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
@@ -44,7 +44,8 @@ import org.apache.nifi.controller.service.mock.MockProcessGroup;
 import org.apache.nifi.engine.FlowEngine;
 import org.apache.nifi.groups.ProcessGroup;
 import org.apache.nifi.logging.ComponentLog;
-import org.apache.nifi.nar.ExtensionManager;
+import org.apache.nifi.nar.ExtensionDiscoveringManager;
+import org.apache.nifi.nar.StandardExtensionDiscoveringManager;
 import org.apache.nifi.nar.SystemBundle;
 import org.apache.nifi.processor.AbstractProcessor;
 import org.apache.nifi.processor.ProcessContext;
@@ -104,6 +105,7 @@ public class TestStandardProcessScheduler {
     private ProcessGroup rootGroup;
     private NiFiProperties nifiProperties;
     private Bundle systemBundle;
+    private ExtensionDiscoveringManager extensionManager;
     private volatile String propsFile = TestStandardProcessScheduler.class.getResource("/standardprocessschedulertest.nifi.properties").getFile();
 
     @Before
@@ -115,7 +117,8 @@ public class TestStandardProcessScheduler {
 
         // load the system bundle
         systemBundle = SystemBundle.create(nifiProperties);
-        ExtensionManager.discoverExtensions(systemBundle, Collections.emptySet());
+        extensionManager = new StandardExtensionDiscoveringManager();
+        extensionManager.discoverExtensions(systemBundle, Collections.emptySet());
 
         scheduler = new StandardProcessScheduler(new FlowEngine(1, "Unit Test", true), Mockito.mock(FlowController.class), null, stateMgrProvider, nifiProperties);
         scheduler.setSchedulingAgent(SchedulingStrategy.TIMER_DRIVEN, Mockito.mock(SchedulingAgent.class));
@@ -130,9 +133,10 @@ public class TestStandardProcessScheduler {
         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,
-            new StandardComponentVariableRegistry(variableRegistry), reloadComponent, new SynchronousValidationTrigger());
+            new StandardComponentVariableRegistry(variableRegistry), reloadComponent, extensionManager, new SynchronousValidationTrigger());
 
         controller = Mockito.mock(FlowController.class);
+        Mockito.when(controller.getExtensionManager()).thenReturn(extensionManager);
 
         final ConcurrentMap<String, ProcessorNode> processorMap = new ConcurrentHashMap<>();
         Mockito.doAnswer(new Answer<ProcessorNode>() {
@@ -205,7 +209,8 @@ public class TestStandardProcessScheduler {
         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, new StandardComponentVariableRegistry(VariableRegistry.EMPTY_REGISTRY), reloadComponent, new SynchronousValidationTrigger());
+            scheduler, serviceProvider, nifiProperties, new StandardComponentVariableRegistry(VariableRegistry.EMPTY_REGISTRY),
+                reloadComponent, extensionManager, new SynchronousValidationTrigger());
         rootGroup.addProcessor(procNode);
 
         Map<String, String> procProps = new HashMap<>();
@@ -507,7 +512,8 @@ public class TestStandardProcessScheduler {
 
         final ProcessorNode procNode = new StandardProcessorNode(loggableComponent, UUID.randomUUID().toString(),
             new StandardValidationContextFactory(controller, variableRegistry),
-            scheduler, controller, nifiProperties, new StandardComponentVariableRegistry(VariableRegistry.EMPTY_REGISTRY), reloadComponent, new SynchronousValidationTrigger());
+            scheduler, controller, nifiProperties, new StandardComponentVariableRegistry(VariableRegistry.EMPTY_REGISTRY),
+                reloadComponent, extensionManager, new SynchronousValidationTrigger());
 
         procNode.performValidation();
         rootGroup.addProcessor(procNode);
@@ -533,7 +539,8 @@ public class TestStandardProcessScheduler {
 
         final ProcessorNode procNode = new StandardProcessorNode(loggableComponent, UUID.randomUUID().toString(),
             new StandardValidationContextFactory(controller, variableRegistry),
-            scheduler, controller, nifiProperties, new StandardComponentVariableRegistry(VariableRegistry.EMPTY_REGISTRY), reloadComponent, new SynchronousValidationTrigger());
+            scheduler, controller, nifiProperties, new StandardComponentVariableRegistry(VariableRegistry.EMPTY_REGISTRY),
+                reloadComponent, extensionManager, new SynchronousValidationTrigger());
 
         rootGroup.addProcessor(procNode);
 
@@ -562,7 +569,8 @@ public class TestStandardProcessScheduler {
 
         final ProcessorNode procNode = new StandardProcessorNode(loggableComponent, UUID.randomUUID().toString(),
             new StandardValidationContextFactory(controller, variableRegistry),
-            scheduler, controller, nifiProperties, new StandardComponentVariableRegistry(VariableRegistry.EMPTY_REGISTRY), reloadComponent, new SynchronousValidationTrigger());
+            scheduler, controller, nifiProperties, new StandardComponentVariableRegistry(VariableRegistry.EMPTY_REGISTRY),
+                reloadComponent, extensionManager, new SynchronousValidationTrigger());
 
         rootGroup.addProcessor(procNode);
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/fdd8cdbb/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/serialization/StandardFlowSerializerTest.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/serialization/StandardFlowSerializerTest.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/serialization/StandardFlowSerializerTest.java
index 91a52a0..f019257 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/serialization/StandardFlowSerializerTest.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/serialization/StandardFlowSerializerTest.java
@@ -16,16 +16,6 @@
  */
 package org.apache.nifi.controller.serialization;
 
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
-import java.io.ByteArrayOutputStream;
-import java.io.File;
-import java.nio.charset.StandardCharsets;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.UUID;
 import org.apache.commons.io.FileUtils;
 import org.apache.nifi.admin.service.AuditService;
 import org.apache.nifi.authorization.AbstractPolicyBasedAuthorizer;
@@ -36,7 +26,8 @@ import org.apache.nifi.controller.FlowController;
 import org.apache.nifi.controller.ProcessorNode;
 import org.apache.nifi.controller.repository.FlowFileEventRepository;
 import org.apache.nifi.encrypt.StringEncryptor;
-import org.apache.nifi.nar.ExtensionManager;
+import org.apache.nifi.nar.ExtensionDiscoveringManager;
+import org.apache.nifi.nar.StandardExtensionDiscoveringManager;
 import org.apache.nifi.nar.SystemBundle;
 import org.apache.nifi.provenance.MockProvenanceRepository;
 import org.apache.nifi.registry.VariableRegistry;
@@ -50,6 +41,17 @@ import org.junit.Test;
 import org.mockito.Mockito;
 import org.w3c.dom.Document;
 
+import java.io.ByteArrayOutputStream;
+import java.io.File;
+import java.nio.charset.StandardCharsets;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.UUID;
+
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
 public class StandardFlowSerializerTest {
 
     private static final String RAW_COMMENTS
@@ -60,6 +62,7 @@ public class StandardFlowSerializerTest {
 
     private FlowController controller;
     private Bundle systemBundle;
+    private ExtensionDiscoveringManager extensionManager;
     private StandardFlowSerializer serializer;
 
     @Before
@@ -78,14 +81,15 @@ public class StandardFlowSerializerTest {
 
         // use the system bundle
         systemBundle = SystemBundle.create(nifiProperties);
-        ExtensionManager.discoverExtensions(systemBundle, Collections.emptySet());
+        extensionManager = new StandardExtensionDiscoveringManager();
+        extensionManager.discoverExtensions(systemBundle, Collections.emptySet());
 
         final AbstractPolicyBasedAuthorizer authorizer = new MockPolicyBasedAuthorizer();
         final VariableRegistry variableRegistry = new FileBasedVariableRegistry(nifiProperties.getVariableRegistryPropertiesPaths());
 
         final BulletinRepository bulletinRepo = Mockito.mock(BulletinRepository.class);
         controller = FlowController.createStandaloneInstance(flowFileEventRepo, nifiProperties, authorizer,
-            auditService, encryptor, bulletinRepo, variableRegistry, Mockito.mock(FlowRegistryClient.class));
+            auditService, encryptor, bulletinRepo, variableRegistry, Mockito.mock(FlowRegistryClient.class), extensionManager);
 
         serializer = new StandardFlowSerializer(encryptor);
     }

http://git-wip-us.apache.org/repos/asf/nifi/blob/fdd8cdbb/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/StandardControllerServiceProviderIT.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/StandardControllerServiceProviderIT.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/StandardControllerServiceProviderIT.java
index e956944..0f4d3ce 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/StandardControllerServiceProviderIT.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/StandardControllerServiceProviderIT.java
@@ -17,16 +17,6 @@
 
 package org.apache.nifi.controller.service;
 
-import static org.junit.Assert.assertTrue;
-
-import java.beans.PropertyDescriptor;
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.LinkedHashMap;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.ExecutionException;
-
 import org.apache.nifi.bundle.Bundle;
 import org.apache.nifi.components.state.StateManager;
 import org.apache.nifi.components.state.StateManagerProvider;
@@ -37,7 +27,8 @@ import org.apache.nifi.controller.service.mock.ServiceA;
 import org.apache.nifi.controller.service.mock.ServiceB;
 import org.apache.nifi.engine.FlowEngine;
 import org.apache.nifi.groups.ProcessGroup;
-import org.apache.nifi.nar.ExtensionManager;
+import org.apache.nifi.nar.ExtensionDiscoveringManager;
+import org.apache.nifi.nar.StandardExtensionDiscoveringManager;
 import org.apache.nifi.nar.SystemBundle;
 import org.apache.nifi.registry.VariableRegistry;
 import org.apache.nifi.util.NiFiProperties;
@@ -46,9 +37,20 @@ import org.junit.BeforeClass;
 import org.junit.Test;
 import org.mockito.Mockito;
 
+import java.beans.PropertyDescriptor;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ExecutionException;
+
+import static org.junit.Assert.assertTrue;
+
 public class StandardControllerServiceProviderIT {
     private static Bundle systemBundle;
     private static NiFiProperties niFiProperties;
+    private static ExtensionDiscoveringManager extensionManager;
     private static VariableRegistry variableRegistry = VariableRegistry.ENVIRONMENT_SYSTEM_REGISTRY;
 
     private static StateManagerProvider stateManagerProvider = new StateManagerProvider() {
@@ -81,7 +83,8 @@ public class StandardControllerServiceProviderIT {
 
         // load the system bundle
         systemBundle = SystemBundle.create(niFiProperties);
-        ExtensionManager.discoverExtensions(systemBundle, Collections.emptySet());
+        extensionManager = new StandardExtensionDiscoveringManager();
+        extensionManager.discoverExtensions(systemBundle, Collections.emptySet());
     }
 
     /**
@@ -102,6 +105,7 @@ public class StandardControllerServiceProviderIT {
         final FlowController controller = Mockito.mock(FlowController.class);
         final ProcessGroup procGroup = new MockProcessGroup(controller);
         Mockito.when(controller.getGroup(Mockito.anyString())).thenReturn(procGroup);
+        Mockito.when(controller.getExtensionManager()).thenReturn(extensionManager);
 
         final StandardControllerServiceProvider provider = new StandardControllerServiceProvider(controller, scheduler, null,
             stateManagerProvider, variableRegistry, niFiProperties, new SynchronousValidationTrigger());

http://git-wip-us.apache.org/repos/asf/nifi/blob/fdd8cdbb/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 f7ee72f..0a0b05f 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
@@ -20,8 +20,9 @@ import org.apache.nifi.bundle.Bundle;
 import org.apache.nifi.components.state.StateManager;
 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.controller.FlowController;
+import org.apache.nifi.nar.ExtensionDiscoveringManager;
+import org.apache.nifi.nar.StandardExtensionDiscoveringManager;
 import org.apache.nifi.nar.SystemBundle;
 import org.apache.nifi.registry.VariableRegistry;
 import org.apache.nifi.registry.variable.FileBasedVariableRegistry;
@@ -33,6 +34,8 @@ import org.junit.BeforeClass;
 import org.junit.Test;
 import org.mockito.Mockito;
 
+import java.util.Collections;
+
 
 public class StandardControllerServiceProviderTest {
 
@@ -40,27 +43,31 @@ public class StandardControllerServiceProviderTest {
     private ControllerService implementation;
     private static VariableRegistry variableRegistry;
     private static NiFiProperties nifiProperties;
+    private static ExtensionDiscoveringManager extensionManager;
     private static Bundle systemBundle;
+    private static FlowController flowController;
 
     @BeforeClass
     public static void setupSuite() throws Exception {
         System.setProperty(NiFiProperties.PROPERTIES_FILE_PATH, StandardControllerServiceProviderTest.class.getResource("/conf/nifi.properties").getFile());
         nifiProperties = NiFiProperties.createBasicNiFiProperties(null, null);
 
-        NarClassLoaders.getInstance().init(nifiProperties.getFrameworkWorkingDirectory(), nifiProperties.getExtensionsWorkingDirectory());
-
         // load the system bundle
         systemBundle = SystemBundle.create(nifiProperties);
-        ExtensionManager.discoverExtensions(systemBundle, NarClassLoaders.getInstance().getBundles());
+        extensionManager = new StandardExtensionDiscoveringManager();
+        extensionManager.discoverExtensions(systemBundle, Collections.emptySet());
 
         variableRegistry = new FileBasedVariableRegistry(nifiProperties.getVariableRegistryPropertiesPaths());
+
+        flowController = Mockito.mock(FlowController.class);
+        Mockito.when(flowController.getExtensionManager()).thenReturn(extensionManager);
     }
 
     @Before
     public void setup() throws Exception {
         String id = "id";
         String clazz = "org.apache.nifi.controller.service.util.TestControllerService";
-        ControllerServiceProvider provider = new StandardControllerServiceProvider(null, null, null, new StateManagerProvider() {
+        ControllerServiceProvider provider = new StandardControllerServiceProvider(flowController, null, null, new StateManagerProvider() {
             @Override
             public StateManager getStateManager(final String componentId) {
                 return Mockito.mock(StateManager.class);

http://git-wip-us.apache.org/repos/asf/nifi/blob/fdd8cdbb/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 b37dbb2..d317000 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
@@ -17,20 +17,6 @@
  */
 package org.apache.nifi.controller.service;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.LinkedHashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.UUID;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.TimeUnit;
-
 import org.apache.nifi.bundle.Bundle;
 import org.apache.nifi.components.state.StateManager;
 import org.apache.nifi.components.state.StateManagerProvider;
@@ -49,7 +35,8 @@ import org.apache.nifi.controller.service.mock.ServiceC;
 import org.apache.nifi.engine.FlowEngine;
 import org.apache.nifi.groups.ProcessGroup;
 import org.apache.nifi.groups.StandardProcessGroup;
-import org.apache.nifi.nar.ExtensionManager;
+import org.apache.nifi.nar.ExtensionDiscoveringManager;
+import org.apache.nifi.nar.StandardExtensionDiscoveringManager;
 import org.apache.nifi.nar.SystemBundle;
 import org.apache.nifi.processor.Processor;
 import org.apache.nifi.processor.StandardValidationContextFactory;
@@ -68,6 +55,20 @@ import org.mockito.Mockito;
 import org.mockito.invocation.InvocationOnMock;
 import org.mockito.stubbing.Answer;
 
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.TimeUnit;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
 public class TestStandardControllerServiceProvider {
 
     private static StateManagerProvider stateManagerProvider = new StateManagerProvider() {
@@ -95,6 +96,7 @@ public class TestStandardControllerServiceProvider {
 
     private static VariableRegistry variableRegistry = VariableRegistry.ENVIRONMENT_SYSTEM_REGISTRY;
     private static NiFiProperties niFiProperties;
+    private static ExtensionDiscoveringManager extensionManager;
     private static Bundle systemBundle;
     private FlowController controller;
 
@@ -105,12 +107,14 @@ public class TestStandardControllerServiceProvider {
 
         // load the system bundle
         systemBundle = SystemBundle.create(niFiProperties);
-        ExtensionManager.discoverExtensions(systemBundle, Collections.emptySet());
+        extensionManager = new StandardExtensionDiscoveringManager();
+        extensionManager.discoverExtensions(systemBundle, Collections.emptySet());
     }
 
     @Before
     public void setup() {
         controller = Mockito.mock(FlowController.class);
+        Mockito.when(controller.getExtensionManager()).thenReturn(extensionManager);
 
         final ConcurrentMap<String, ProcessorNode> processorMap = new ConcurrentHashMap<>();
         Mockito.doAnswer(new Answer<ProcessorNode>() {
@@ -146,6 +150,7 @@ public class TestStandardControllerServiceProvider {
         final ProcessGroup procGroup = new MockProcessGroup(controller);
         final FlowController controller = Mockito.mock(FlowController.class);
         Mockito.when(controller.getGroup(Mockito.anyString())).thenReturn(procGroup);
+        Mockito.when(controller.getExtensionManager()).thenReturn(extensionManager);
 
         final StandardProcessScheduler scheduler = createScheduler();
         final StandardControllerServiceProvider provider =
@@ -164,6 +169,7 @@ public class TestStandardControllerServiceProvider {
         final ProcessGroup group = new MockProcessGroup(controller);
         final FlowController controller = Mockito.mock(FlowController.class);
         Mockito.when(controller.getGroup(Mockito.anyString())).thenReturn(group);
+        Mockito.when(controller.getExtensionManager()).thenReturn(extensionManager);
 
         final StandardProcessScheduler scheduler = createScheduler();
         final StandardControllerServiceProvider provider =
@@ -219,6 +225,7 @@ public class TestStandardControllerServiceProvider {
         final ProcessGroup procGroup = new MockProcessGroup(controller);
         final FlowController controller = Mockito.mock(FlowController.class);
         Mockito.when(controller.getGroup(Mockito.anyString())).thenReturn(procGroup);
+        Mockito.when(controller.getExtensionManager()).thenReturn(extensionManager);
 
         final StandardControllerServiceProvider provider =
             new StandardControllerServiceProvider(controller, null, null, stateManagerProvider, variableRegistry, niFiProperties, new SynchronousValidationTrigger());
@@ -379,7 +386,7 @@ public class TestStandardControllerServiceProvider {
         final LoggableComponent<Processor> dummyProcessor = new LoggableComponent<>(processor, systemBundle.getBundleDetails().getCoordinate(), null);
         final ProcessorNode procNode = new StandardProcessorNode(dummyProcessor, mockInitContext.getIdentifier(),
                 new StandardValidationContextFactory(serviceProvider, null), scheduler, serviceProvider, niFiProperties,
-            new StandardComponentVariableRegistry(VariableRegistry.EMPTY_REGISTRY), reloadComponent, new SynchronousValidationTrigger());
+            new StandardComponentVariableRegistry(VariableRegistry.EMPTY_REGISTRY), reloadComponent, extensionManager, new SynchronousValidationTrigger());
 
         final ProcessGroup group = new StandardProcessGroup(UUID.randomUUID().toString(), serviceProvider, scheduler, null, null, Mockito.mock(FlowController.class),
             new MutableVariableRegistry(variableRegistry));
@@ -394,6 +401,7 @@ public class TestStandardControllerServiceProvider {
         final ProcessGroup procGroup = new MockProcessGroup(controller);
         final FlowController controller = Mockito.mock(FlowController.class);
         Mockito.when(controller.getGroup(Mockito.anyString())).thenReturn(procGroup);
+        Mockito.when(controller.getExtensionManager()).thenReturn(extensionManager);
 
         final StandardProcessScheduler scheduler = createScheduler();
         final StandardControllerServiceProvider provider =
@@ -421,6 +429,7 @@ public class TestStandardControllerServiceProvider {
             new StandardControllerServiceProvider(controller, scheduler, null, stateManagerProvider, variableRegistry, niFiProperties, new SynchronousValidationTrigger());
         ProcessGroup procGroup = new MockProcessGroup(controller);
         Mockito.when(controller.getGroup(Mockito.anyString())).thenReturn(procGroup);
+        Mockito.when(controller.getExtensionManager()).thenReturn(extensionManager);
 
         ControllerServiceNode A = provider.createControllerService(ServiceA.class.getName(), "A",
                 systemBundle.getBundleDetails().getCoordinate(), null, false);
@@ -474,6 +483,7 @@ public class TestStandardControllerServiceProvider {
             stateManagerProvider, variableRegistry, niFiProperties, new SynchronousValidationTrigger());
         ProcessGroup procGroup = new MockProcessGroup(controller);
         Mockito.when(controller.getGroup(Mockito.anyString())).thenReturn(procGroup);
+        Mockito.when(controller.getExtensionManager()).thenReturn(extensionManager);
 
         ControllerServiceNode A = provider.createControllerService(ServiceC.class.getName(), "A",
                 systemBundle.getBundleDetails().getCoordinate(), null, false);
@@ -519,6 +529,7 @@ public class TestStandardControllerServiceProvider {
             new StandardControllerServiceProvider(controller, scheduler, null, stateManagerProvider, variableRegistry, niFiProperties, new SynchronousValidationTrigger());
         ProcessGroup procGroup = new MockProcessGroup(controller);
         Mockito.when(controller.getGroup(Mockito.anyString())).thenReturn(procGroup);
+        Mockito.when(controller.getExtensionManager()).thenReturn(extensionManager);
 
         ControllerServiceNode serviceNode1 = provider.createControllerService(ServiceA.class.getName(), "1",
                 systemBundle.getBundleDetails().getCoordinate(), null, false);

http://git-wip-us.apache.org/repos/asf/nifi/blob/fdd8cdbb/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/fingerprint/FingerprintFactoryTest.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/fingerprint/FingerprintFactoryTest.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/fingerprint/FingerprintFactoryTest.java
index 30294a4..09acbc9 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/fingerprint/FingerprintFactoryTest.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/fingerprint/FingerprintFactoryTest.java
@@ -43,6 +43,8 @@ import org.apache.nifi.controller.serialization.ScheduledStateLookup;
 import org.apache.nifi.controller.serialization.StandardFlowSerializer;
 import org.apache.nifi.encrypt.StringEncryptor;
 import org.apache.nifi.groups.RemoteProcessGroup;
+import org.apache.nifi.nar.ExtensionManager;
+import org.apache.nifi.nar.StandardExtensionDiscoveringManager;
 import org.apache.nifi.remote.RemoteGroupPort;
 import org.apache.nifi.remote.protocol.SiteToSiteTransportProtocol;
 import org.junit.Before;
@@ -58,12 +60,14 @@ import org.xml.sax.SAXParseException;
 public class FingerprintFactoryTest {
 
     private StringEncryptor encryptor;
+    private ExtensionManager extensionManager;
     private FingerprintFactory fingerprinter;
 
     @Before
     public void setup() {
         encryptor = new StringEncryptor("PBEWITHMD5AND256BITAES-CBC-OPENSSL", "BC", "nififtw!");
-        fingerprinter = new FingerprintFactory(encryptor);
+        extensionManager = new StandardExtensionDiscoveringManager();
+        fingerprinter = new FingerprintFactory(encryptor, extensionManager);
     }
 
     @Test
@@ -122,7 +126,7 @@ public class FingerprintFactoryTest {
 
     @Test
     public void testSchemaValidation() throws IOException {
-        FingerprintFactory fp = new FingerprintFactory(null, getValidatingDocumentBuilder());
+        FingerprintFactory fp = new FingerprintFactory(null, getValidatingDocumentBuilder(), extensionManager);
         final String fingerprint = fp.createFingerprint(getResourceBytes("/nifi/fingerprint/validating-flow.xml"), null);
     }
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/fdd8cdbb/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-loading-utils/pom.xml
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-loading-utils/pom.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-loading-utils/pom.xml
new file mode 100644
index 0000000..1d4e2de
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-loading-utils/pom.xml
@@ -0,0 +1,46 @@
+<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.9.0-SNAPSHOT</version>
+    </parent>
+    <artifactId>nifi-framework-nar-loading-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-documentation</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/fdd8cdbb/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-loading-utils/src/main/java/org/apache/nifi/nar/ExtensionUiLoader.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-loading-utils/src/main/java/org/apache/nifi/nar/ExtensionUiLoader.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-loading-utils/src/main/java/org/apache/nifi/nar/ExtensionUiLoader.java
new file mode 100644
index 0000000..0c5b02b
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-loading-utils/src/main/java/org/apache/nifi/nar/ExtensionUiLoader.java
@@ -0,0 +1,33 @@
+/*
+ * 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 java.util.Set;
+
+/**
+ * Loads WARs from extensions and makes them available to the running application.
+ */
+public interface ExtensionUiLoader {
+
+    /**
+     * @param bundles the set of bundles to load WARs from
+     */
+    void loadExtensionUis(Set<Bundle> bundles);
+
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/fdd8cdbb/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-loading-utils/src/main/java/org/apache/nifi/nar/NarAutoLoader.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-loading-utils/src/main/java/org/apache/nifi/nar/NarAutoLoader.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-loading-utils/src/main/java/org/apache/nifi/nar/NarAutoLoader.java
new file mode 100644
index 0000000..2ac55a0
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-loading-utils/src/main/java/org/apache/nifi/nar/NarAutoLoader.java
@@ -0,0 +1,87 @@
+/*
+ * 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.FileUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.FileSystems;
+import java.nio.file.Path;
+import java.nio.file.StandardWatchEventKinds;
+import java.nio.file.WatchService;
+import java.util.Objects;
+
+/**
+ * Starts a thread to monitor the auto-load directory for new NARs.
+ */
+public class NarAutoLoader {
+
+    private static final Logger LOGGER = LoggerFactory.getLogger(NarAutoLoader.class);
+
+    private static final long POLL_INTERVAL_MS = 5000;
+
+    private final File autoLoadDir;
+    private final NarLoader narLoader;
+
+    private volatile NarAutoLoaderTask narAutoLoaderTask;
+    private volatile boolean started = false;
+
+    public NarAutoLoader(final File autoLoadDir, final NarLoader narLoader) {
+        this.autoLoadDir = Objects.requireNonNull(autoLoadDir);
+        this.narLoader = Objects.requireNonNull(narLoader);
+    }
+
+    public synchronized void start() throws IOException {
+        if (started) {
+            return;
+        }
+
+        FileUtils.ensureDirectoryExistAndCanReadAndWrite(autoLoadDir);
+
+        final WatchService watcher = FileSystems.getDefault().newWatchService();
+
+        final Path autoLoadPath = autoLoadDir.toPath();
+        autoLoadPath.register(watcher, StandardWatchEventKinds.ENTRY_CREATE);
+
+        narAutoLoaderTask = new NarAutoLoaderTask.Builder()
+                .autoLoadPath(autoLoadPath)
+                .watchService(watcher)
+                .pollIntervalMillis(POLL_INTERVAL_MS)
+                .narLoader(narLoader)
+                .build();
+
+        LOGGER.info("Starting NAR Auto-Loader for directory {} ...", new Object[]{autoLoadPath});
+
+        final Thread thread = new Thread(narAutoLoaderTask);
+        thread.setName("NAR Auto-Loader");
+        thread.setDaemon(true);
+        thread.start();
+
+        LOGGER.info("NAR Auto-Loader started");
+        started = true;
+    }
+
+    public synchronized void stop() {
+        started = false;
+        narAutoLoaderTask.stop();
+        LOGGER.info("NAR Auto-Loader stopped");
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/fdd8cdbb/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-loading-utils/src/main/java/org/apache/nifi/nar/NarAutoLoaderTask.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-loading-utils/src/main/java/org/apache/nifi/nar/NarAutoLoaderTask.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-loading-utils/src/main/java/org/apache/nifi/nar/NarAutoLoaderTask.java
new file mode 100644
index 0000000..260fe10
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-loading-utils/src/main/java/org/apache/nifi/nar/NarAutoLoaderTask.java
@@ -0,0 +1,178 @@
+/*
+ * 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.File;
+import java.nio.file.Path;
+import java.nio.file.StandardWatchEventKinds;
+import java.nio.file.WatchEvent;
+import java.nio.file.WatchKey;
+import java.nio.file.WatchService;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * The runnable task that polls the WatchService for new NAR files found in the auto-load directory.
+ *
+ * Each new NAR file found will be passed to the NarLoader to be unpacked and loaded into the ExtensionManager.
+ *
+ */
+public class NarAutoLoaderTask implements Runnable {
+
+    private static final Logger LOGGER = LoggerFactory.getLogger(NarAutoLoaderTask.class);
+
+    private static final long MIN_FILE_AGE = 5000;
+
+    private final Path autoLoadPath;
+    private final WatchService watchService;
+    private final long pollIntervalMillis;
+    private final NarLoader narLoader;
+    private final List<File> candidateNars;
+
+    private volatile boolean stopped = false;
+
+    private NarAutoLoaderTask(final Builder builder) {
+        this.autoLoadPath = builder.autoLoadPath;
+        this.watchService = builder.watchService;
+        this.pollIntervalMillis = builder.pollIntervalMillis;
+        this.narLoader = builder.narLoader;
+        this.candidateNars = new ArrayList<>();
+    }
+
+    @Override
+    public void run() {
+        while (!stopped) {
+            try {
+                WatchKey key;
+                try {
+                    LOGGER.debug("Polling for new NARs at {}", new Object[]{autoLoadPath});
+                    key = watchService.poll(pollIntervalMillis, TimeUnit.MILLISECONDS);
+                } catch (InterruptedException x) {
+                    LOGGER.info("WatchService interrupted, returning...");
+                    return;
+                }
+
+                // Key comes back as null when there are no new create events, but we still want to continue processing
+                // so we can consider files added to the candidateNars list in previous iterations
+
+                if (key != null) {
+                    for (WatchEvent<?> event : key.pollEvents()) {
+                        final WatchEvent.Kind<?> kind = event.kind();
+                        if (kind == StandardWatchEventKinds.OVERFLOW) {
+                            continue;
+                        }
+
+                        final WatchEvent<Path> ev = (WatchEvent<Path>) event;
+                        final Path filename = ev.context();
+
+                        final Path autoLoadFile = autoLoadPath.resolve(filename);
+                        final String autoLoadFilename = autoLoadFile.toFile().getName().toLowerCase();
+
+                        if (!autoLoadFilename.endsWith(".nar")) {
+                            LOGGER.info("Skipping non-nar file {}", new Object[]{autoLoadFilename});
+                            continue;
+                        }
+
+                        if (autoLoadFilename.startsWith(".")) {
+                            LOGGER.debug("Skipping partially written file {}", new Object[]{autoLoadFilename});
+                            continue;
+                        }
+
+                        LOGGER.info("Found {} in auto-load directory", new Object[]{autoLoadFile});
+                        candidateNars.add(autoLoadFile.toFile());
+                    }
+
+                    final boolean valid = key.reset();
+                    if (!valid) {
+                        LOGGER.error("NAR auto-load directory is no longer valid");
+                        stop();
+                    }
+                }
+
+                // Make sure that the created file is done being written by checking the last modified date of the file and
+                // make sure a certain amount of time has passed indicating it is done being written to
+
+                final List<File> readyNars = new ArrayList<>();
+                final Iterator<File> candidateNarIter = candidateNars.iterator();
+                while (candidateNarIter.hasNext()) {
+                    final File candidateNar = candidateNarIter.next();
+                    final long fileAge = System.currentTimeMillis() - candidateNar.lastModified();
+                    if (fileAge >= MIN_FILE_AGE) {
+                        readyNars.add(candidateNar);
+                        candidateNarIter.remove();
+                    } else {
+                        LOGGER.debug("Candidate NAR {} not ready yet, will check again next time", new Object[]{candidateNar.getName()});
+                    }
+                }
+
+                if (!readyNars.isEmpty()) {
+                    narLoader.load(readyNars);
+                }
+
+            } catch (final Exception e) {
+                LOGGER.error("Error loading NARs due to: " + e.getMessage(), e);
+            }
+        }
+    }
+
+    public void stop() {
+        LOGGER.info("Stopping NAR Auto-loader");
+        stopped = true;
+    }
+
+    /**
+     * Builder for NarAutoLoaderTask.
+     */
+    public static class Builder {
+
+        private Path autoLoadPath;
+        private WatchService watchService;
+        private long pollIntervalMillis;
+        private NarLoader narLoader;
+
+        public Builder autoLoadPath(final Path autoLoadPath) {
+            this.autoLoadPath = autoLoadPath;
+            return this;
+        }
+
+        public Builder watchService(final WatchService watchService) {
+            this.watchService = watchService;
+            return this;
+        }
+
+        public Builder pollIntervalMillis(final long pollIntervalMillis) {
+            this.pollIntervalMillis = pollIntervalMillis;
+            return this;
+        }
+
+        public Builder narLoader(final NarLoader narLoader) {
+            this.narLoader = narLoader;
+            return this;
+        }
+
+        public NarAutoLoaderTask build() {
+            return new NarAutoLoaderTask(this);
+        }
+
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/fdd8cdbb/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-loading-utils/src/main/java/org/apache/nifi/nar/NarLoader.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-loading-utils/src/main/java/org/apache/nifi/nar/NarLoader.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-loading-utils/src/main/java/org/apache/nifi/nar/NarLoader.java
new file mode 100644
index 0000000..6aa4d43
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-loading-utils/src/main/java/org/apache/nifi/nar/NarLoader.java
@@ -0,0 +1,35 @@
+/*
+ * 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.File;
+import java.util.Collection;
+
+/**
+ * Responsible for loading NARs into the running application.
+ */
+public interface NarLoader {
+
+    /**
+     * Loads the given NARs.
+     *
+     * @param narFiles the collection of NAR files to load
+     * @return the result of loading the NARs (i.e. bundles that were loaded, bundles that were skipped)
+     */
+    NarLoadResult load(final Collection<File> narFiles);
+
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/fdd8cdbb/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-loading-utils/src/main/java/org/apache/nifi/nar/StandardNarLoader.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-loading-utils/src/main/java/org/apache/nifi/nar/StandardNarLoader.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-loading-utils/src/main/java/org/apache/nifi/nar/StandardNarLoader.java
new file mode 100644
index 0000000..2ddca01
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-loading-utils/src/main/java/org/apache/nifi/nar/StandardNarLoader.java
@@ -0,0 +1,167 @@
+/*
+ * 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.documentation.DocGenerator;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.jar.Attributes;
+import java.util.jar.JarFile;
+import java.util.jar.Manifest;
+
+/**
+ * Loads a set of NARs from the file system into the running application.
+ *
+ * NOTE: Initially this will only be used from the NarAutoLoader which is watching a directory for new files, but eventually
+ * this may also be used for loading a NAR that was downloaded from the extension registry, and thus the load method
+ * is synchronized to ensure only one set of NARs can be in process of loading at a given time.
+ */
+public class StandardNarLoader implements NarLoader {
+
+    private static Logger LOGGER = LoggerFactory.getLogger(StandardNarLoader.class);
+
+    private final File extensionsWorkingDir;
+    private final File docsWorkingDir;
+    private final NarClassLoaders narClassLoaders;
+    private final ExtensionDiscoveringManager extensionManager;
+    private final ExtensionMapping extensionMapping;
+    private final ExtensionUiLoader extensionUiLoader;
+
+    private Set<BundleDetails> previouslySkippedBundles;
+
+    public StandardNarLoader(final File extensionsWorkingDir,
+                             final File docsWorkingDir,
+                             final NarClassLoaders narClassLoaders,
+                             final ExtensionDiscoveringManager extensionManager,
+                             final ExtensionMapping extensionMapping,
+                             final ExtensionUiLoader extensionUiLoader) {
+        this.extensionsWorkingDir = extensionsWorkingDir;
+        this.docsWorkingDir = docsWorkingDir;
+        this.narClassLoaders = narClassLoaders;
+        this.extensionManager = extensionManager;
+        this.extensionMapping = extensionMapping;
+        this.extensionUiLoader = extensionUiLoader;
+    }
+
+    @Override
+    public synchronized NarLoadResult load(final Collection<File> narFiles) {
+        LOGGER.info("Starting load process for {} NARs...", new Object[]{narFiles.size()});
+
+        final List<File> unpackedNars = new ArrayList<>();
+
+        for (final File narFile : narFiles) {
+            LOGGER.debug("Unpacking {}...", new Object[]{narFile.getName()});
+            final File unpackedNar = unpack(narFile);
+            if (unpackedNar != null) {
+                LOGGER.debug("Completed unpacking {}", new Object[]{narFile.getName()});
+                unpackedNars.add(unpackedNar);
+            }
+        }
+
+        if (previouslySkippedBundles != null && !previouslySkippedBundles.isEmpty()) {
+            LOGGER.info("Including {} previously skipped bundle(s)", new Object[]{previouslySkippedBundles.size()});
+            previouslySkippedBundles.forEach(b -> unpackedNars.add(b.getWorkingDirectory()));
+        }
+
+        if (unpackedNars.isEmpty()) {
+            LOGGER.info("No NARs were unpacked, nothing to do");
+            return new NarLoadResult(Collections.emptySet(), Collections.emptySet());
+        }
+
+        LOGGER.info("Creating class loaders for {} NARs...", new Object[]{unpackedNars.size()});
+
+        final NarLoadResult narLoadResult = narClassLoaders.loadAdditionalNars(unpackedNars);
+        final Set<Bundle> loadedBundles = narLoadResult.getLoadedBundles();
+        final Set<BundleDetails> skippedBundles = narLoadResult.getSkippedBundles();
+
+        LOGGER.info("Successfully created class loaders for {} NARs, {} were skipped",
+                new Object[]{loadedBundles.size(), skippedBundles.size()});
+
+        // Store skipped bundles for next iteration
+        previouslySkippedBundles = new HashSet<>(skippedBundles);
+
+        if (!loadedBundles.isEmpty()) {
+            LOGGER.debug("Discovering extensions...");
+            extensionManager.discoverExtensions(loadedBundles);
+
+            // Call the DocGenerator for the classes that were loaded from each Bundle
+            for (final Bundle bundle : loadedBundles) {
+                final BundleCoordinate bundleCoordinate = bundle.getBundleDetails().getCoordinate();
+                final Set<Class> extensions = extensionManager.getTypes(bundleCoordinate);
+                if (extensions.isEmpty()) {
+                    LOGGER.debug("No documentation to generate for {} because no extensions were found",
+                            new Object[]{bundleCoordinate.getCoordinate()});
+                } else {
+                    LOGGER.debug("Generating documentation for {} extensions in {}",
+                            new Object[]{extensions.size(), bundleCoordinate.getCoordinate()});
+                    DocGenerator.documentConfigurableComponent(extensions, docsWorkingDir, extensionManager);
+                }
+            }
+
+            LOGGER.debug("Loading custom UIs for extensions...");
+            extensionUiLoader.loadExtensionUis(loadedBundles);
+        }
+
+        LOGGER.info("Finished NAR loading process!");
+        return narLoadResult;
+    }
+
+    private File unpack(final File narFile) {
+        try (final JarFile nar = new JarFile(narFile)) {
+            final Manifest manifest = nar.getManifest();
+
+            final Attributes attributes = manifest.getMainAttributes();
+            final String groupId = attributes.getValue(NarManifestEntry.NAR_GROUP.getManifestName());
+            final String narId = attributes.getValue(NarManifestEntry.NAR_ID.getManifestName());
+            final String version = attributes.getValue(NarManifestEntry.NAR_VERSION.getManifestName());
+
+            if (NarClassLoaders.FRAMEWORK_NAR_ID.equals(narId)) {
+                LOGGER.error("Found a framework NAR, will not load {}", new Object[]{narFile.getAbsolutePath()});
+                return null;
+            }
+
+            final BundleCoordinate coordinate = new BundleCoordinate(groupId, narId, version);
+
+            final Bundle bundle = extensionManager.getBundle(coordinate);
+            if (bundle != null) {
+                LOGGER.warn("Found existing bundle with coordinate {}, will not load {}",
+                        new Object[]{coordinate, narFile.getAbsolutePath()});
+                return null;
+            }
+
+            final File unpackedExtension = NarUnpacker.unpackNar(narFile, extensionsWorkingDir);
+            NarUnpacker.mapExtension(unpackedExtension, coordinate, docsWorkingDir, extensionMapping);
+            return unpackedExtension;
+
+        } catch (Exception e) {
+            LOGGER.error("Error unpacking " + narFile.getAbsolutePath(), e);
+            return null;
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/fdd8cdbb/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-loading-utils/src/test/java/org/apache/nifi/nar/TestNarLoader.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-loading-utils/src/test/java/org/apache/nifi/nar/TestNarLoader.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-loading-utils/src/test/java/org/apache/nifi/nar/TestNarLoader.java
new file mode 100644
index 0000000..66cb829
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-loading-utils/src/test/java/org/apache/nifi/nar/TestNarLoader.java
@@ -0,0 +1,189 @@
+/*
+ * 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.controller.ControllerService;
+import org.apache.nifi.processor.Processor;
+import org.apache.nifi.reporting.ReportingTask;
+import org.apache.nifi.util.NiFiProperties;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.File;
+import java.io.IOException;
+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.StandardCopyOption;
+import java.nio.file.attribute.BasicFileAttributes;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+
+public class TestNarLoader {
+
+    static final String WORK_DIR = "./target/work";
+    static final String NAR_AUTOLOAD_DIR = "./target/extensions";
+    static final String EXTENSIONS_DIR = "./src/test/resources/extensions";
+
+    private NiFiProperties properties;
+    private ExtensionMapping extensionMapping;
+
+    private StandardNarLoader narLoader;
+    private NarClassLoaders narClassLoaders;
+    private ExtensionDiscoveringManager extensionManager;
+
+    @Before
+    public void setup() throws IOException, ClassNotFoundException {
+        deleteDir(WORK_DIR);
+        deleteDir(NAR_AUTOLOAD_DIR);
+
+        final File extensionsDir = new File(NAR_AUTOLOAD_DIR);
+        assertTrue(extensionsDir.mkdirs());
+
+        // Create NiFiProperties
+        final String propertiesFile = "./src/test/resources/conf/nifi.properties";
+        properties = NiFiProperties.createBasicNiFiProperties(propertiesFile , Collections.emptyMap());
+
+        // Unpack NARs
+        final Bundle systemBundle = SystemBundle.create(properties);
+        extensionMapping = NarUnpacker.unpackNars(properties, systemBundle);
+        assertEquals(0, extensionMapping.getAllExtensionNames().size());
+
+        // Initialize NarClassLoaders
+        narClassLoaders = new NarClassLoaders();
+        narClassLoaders.init(properties.getFrameworkWorkingDirectory(), properties.getExtensionsWorkingDirectory());
+
+        extensionManager = new StandardExtensionDiscoveringManager();
+        extensionManager.discoverExtensions(systemBundle, narClassLoaders.getBundles());
+
+        // Should have Framework and Jetty NARs loaded here
+        assertEquals(2, narClassLoaders.getBundles().size());
+
+        // No extensions should be loaded yet
+        assertEquals(0, extensionManager.getExtensions(Processor.class).size());
+        assertEquals(0, extensionManager.getExtensions(ControllerService.class).size());
+        assertEquals(0, extensionManager.getExtensions(ReportingTask.class).size());
+
+        // Create class we are testing
+        narLoader = new StandardNarLoader(
+                properties.getExtensionsWorkingDirectory(),
+                properties.getComponentDocumentationWorkingDirectory(),
+                narClassLoaders,
+                extensionManager,
+                extensionMapping,
+                (bundles) -> {});
+    }
+
+    @Test
+    public void testNarLoaderWhenAllAvailable() throws IOException {
+        // Copy all NARs from src/test/resources/extensions to target/extensions
+        final File extensionsDir = new File(EXTENSIONS_DIR);
+        final Path narAutoLoadDir = Paths.get(NAR_AUTOLOAD_DIR);
+        for (final File extensionFile : extensionsDir.listFiles()) {
+            Files.copy(extensionFile.toPath(), narAutoLoadDir.resolve(extensionFile.getName()), StandardCopyOption.REPLACE_EXISTING);
+        }
+
+        final List<File> narFiles = Arrays.asList(narAutoLoadDir.toFile().listFiles());
+        assertEquals(3, narFiles.size());
+
+        final NarLoadResult narLoadResult = narLoader.load(narFiles);
+        assertNotNull(narLoadResult);
+        assertEquals(3, narLoadResult.getLoadedBundles().size());
+        assertEquals(0, narLoadResult.getSkippedBundles().size());
+
+        assertEquals(5, narClassLoaders.getBundles().size());
+        assertEquals(1, extensionManager.getExtensions(Processor.class).size());
+        assertEquals(1, extensionManager.getExtensions(ControllerService.class).size());
+        assertEquals(0, extensionManager.getExtensions(ReportingTask.class).size());
+    }
+
+    @Test
+    public void testNarLoaderWhenDependentNarsAreMissing() throws IOException {
+        final File extensionsDir = new File(EXTENSIONS_DIR);
+        final Path narAutoLoadDir = Paths.get(NAR_AUTOLOAD_DIR);
+
+        // Copy processors NAR first which depends on service API NAR
+        final File processorsNar = new File(extensionsDir, "nifi-example-processors-nar-1.0.nar");
+        final File targetProcessorNar = new File(narAutoLoadDir.toFile(), processorsNar.getName());
+        Files.copy(processorsNar.toPath(), targetProcessorNar.toPath(), StandardCopyOption.REPLACE_EXISTING);
+
+        // Attempt to load while only processor NAR is available
+        final List<File> narFiles1 = Arrays.asList(targetProcessorNar);
+        final NarLoadResult narLoadResult1 = narLoader.load(narFiles1);
+        assertNotNull(narLoadResult1);
+        assertEquals(0, narLoadResult1.getLoadedBundles().size());
+        assertEquals(1, narLoadResult1.getSkippedBundles().size());
+
+        // Copy the service impl which also depends on service API NAR
+        final File serviceImplNar = new File(extensionsDir, "nifi-example-service-nar-1.1.nar");
+        final File targetServiceImplNar = new File(narAutoLoadDir.toFile(), serviceImplNar.getName());
+        Files.copy(serviceImplNar.toPath(), targetServiceImplNar.toPath(), StandardCopyOption.REPLACE_EXISTING);
+
+        // Attempt to load while processor and service impl NARs available
+        final List<File> narFiles2 = Arrays.asList(targetServiceImplNar);
+        final NarLoadResult narLoadResult2 = narLoader.load(narFiles2);
+        assertNotNull(narLoadResult2);
+        assertEquals(0, narLoadResult2.getLoadedBundles().size());
+        assertEquals(2, narLoadResult2.getSkippedBundles().size());
+
+        // Copy service API NAR
+        final File serviceApiNar = new File(extensionsDir, "nifi-example-service-api-nar-1.0.nar");
+        final File targetServiceApiNar = new File(narAutoLoadDir.toFile(), serviceApiNar.getName());
+        Files.copy(serviceApiNar.toPath(), targetServiceApiNar.toPath(), StandardCopyOption.REPLACE_EXISTING);
+
+        // Attempt to load while all NARs available
+        final List<File> narFiles3 = Arrays.asList(targetServiceApiNar);
+        final NarLoadResult narLoadResult3 = narLoader.load(narFiles3);
+        assertNotNull(narLoadResult3);
+        assertEquals(3, narLoadResult3.getLoadedBundles().size());
+        assertEquals(0, narLoadResult3.getSkippedBundles().size());
+
+        assertEquals(5, narClassLoaders.getBundles().size());
+        assertEquals(1, extensionManager.getExtensions(Processor.class).size());
+        assertEquals(1, extensionManager.getExtensions(ControllerService.class).size());
+        assertEquals(0, extensionManager.getExtensions(ReportingTask.class).size());
+    }
+
+    private void deleteDir(String path) throws IOException {
+        Path directory = Paths.get(path);
+        if (!directory.toFile().exists()) {
+            return;
+        }
+
+        Files.walkFileTree(directory, new SimpleFileVisitor<Path>() {
+            @Override
+            public FileVisitResult visitFile(Path file, BasicFileAttributes attrs) throws IOException {
+                Files.delete(file);
+                return FileVisitResult.CONTINUE;
+            }
+
+            @Override
+            public FileVisitResult postVisitDirectory(Path dir, IOException exc) throws IOException {
+                Files.delete(dir);
+                return FileVisitResult.CONTINUE;
+            }
+        });
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/fdd8cdbb/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-loading-utils/src/test/resources/README
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-loading-utils/src/test/resources/README b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-loading-utils/src/test/resources/README
new file mode 100644
index 0000000..1782c2f
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-loading-utils/src/test/resources/README
@@ -0,0 +1,9 @@
+The NARs in src/test/resources/extensions came from building the source code found in nifi-example-nars-source.zip.
+
+If you need to make a change to these NARs, perform the following steps:
+
+1) Copy nifi-example-nars-source.zip somewhere and extract it
+2) Modify code and build
+3) Replace NARs in src/test/resources/extensions with results from build in step 2
+4) Create a zip archive of the modified source code
+5) Replace nifi-example-nars-source.zip with the new zip archive
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi/blob/fdd8cdbb/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-loading-utils/src/test/resources/conf/nifi.properties
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-loading-utils/src/test/resources/conf/nifi.properties b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-loading-utils/src/test/resources/conf/nifi.properties
new file mode 100644
index 0000000..987a550
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-loading-utils/src/test/resources/conf/nifi.properties
@@ -0,0 +1,124 @@
+# 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=./src/test/resources/lib/
+nifi.nar.library.autoload.directory=./target/extensions
+
+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.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/fdd8cdbb/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-loading-utils/src/test/resources/extensions/nifi-example-processors-nar-1.0.nar
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-loading-utils/src/test/resources/extensions/nifi-example-processors-nar-1.0.nar b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-loading-utils/src/test/resources/extensions/nifi-example-processors-nar-1.0.nar
new file mode 100644
index 0000000..317024a
Binary files /dev/null and b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-loading-utils/src/test/resources/extensions/nifi-example-processors-nar-1.0.nar differ

http://git-wip-us.apache.org/repos/asf/nifi/blob/fdd8cdbb/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-loading-utils/src/test/resources/extensions/nifi-example-service-api-nar-1.0.nar
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-loading-utils/src/test/resources/extensions/nifi-example-service-api-nar-1.0.nar b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-loading-utils/src/test/resources/extensions/nifi-example-service-api-nar-1.0.nar
new file mode 100644
index 0000000..a540b2d
Binary files /dev/null and b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-loading-utils/src/test/resources/extensions/nifi-example-service-api-nar-1.0.nar differ

http://git-wip-us.apache.org/repos/asf/nifi/blob/fdd8cdbb/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-loading-utils/src/test/resources/extensions/nifi-example-service-nar-1.1.nar
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-loading-utils/src/test/resources/extensions/nifi-example-service-nar-1.1.nar b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-loading-utils/src/test/resources/extensions/nifi-example-service-nar-1.1.nar
new file mode 100644
index 0000000..82f02a0
Binary files /dev/null and b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-loading-utils/src/test/resources/extensions/nifi-example-service-nar-1.1.nar differ

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

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

http://git-wip-us.apache.org/repos/asf/nifi/blob/fdd8cdbb/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-loading-utils/src/test/resources/nifi-example-nars-source.zip
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-loading-utils/src/test/resources/nifi-example-nars-source.zip b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-loading-utils/src/test/resources/nifi-example-nars-source.zip
new file mode 100644
index 0000000..e4477a5
Binary files /dev/null and b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-loading-utils/src/test/resources/nifi-example-nars-source.zip differ

http://git-wip-us.apache.org/repos/asf/nifi/blob/fdd8cdbb/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
index f6ab922..61d1073 100644
--- 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
@@ -18,6 +18,7 @@ package org.apache.nifi.init;
 
 import org.apache.nifi.components.ConfigurableComponent;
 import org.apache.nifi.controller.ControllerService;
+import org.apache.nifi.nar.ExtensionManager;
 import org.apache.nifi.processor.Processor;
 import org.apache.nifi.reporting.ReportingTask;
 
@@ -30,13 +31,14 @@ public class ConfigurableComponentInitializerFactory {
      * @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) {
+    public static ConfigurableComponentInitializer createComponentInitializer(
+            final ExtensionManager extensionManager, final Class<? extends ConfigurableComponent> componentClass) {
         if (Processor.class.isAssignableFrom(componentClass)) {
-            return new ProcessorInitializer();
+            return new ProcessorInitializer(extensionManager);
         } else if (ControllerService.class.isAssignableFrom(componentClass)) {
-            return new ControllerServiceInitializer();
+            return new ControllerServiceInitializer(extensionManager);
         } else if (ReportingTask.class.isAssignableFrom(componentClass)) {
-            return new ReportingTaskingInitializer();
+            return new ReportingTaskingInitializer(extensionManager);
         }
 
         return null;

http://git-wip-us.apache.org/repos/asf/nifi/blob/fdd8cdbb/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
index 21b107f..1c3f136 100644
--- 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
@@ -35,25 +35,31 @@ import org.apache.nifi.reporting.InitializationException;
  */
 public class ControllerServiceInitializer implements ConfigurableComponentInitializer {
 
+    private final ExtensionManager extensionManager;
+
+    public ControllerServiceInitializer(final ExtensionManager extensionManager) {
+        this.extensionManager = extensionManager;
+    }
+
     @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())) {
+        try (NarCloseable narCloseable = NarCloseable.withComponentNarLoader(extensionManager, component.getClass(), context.getIdentifier())) {
             controllerService.initialize(context);
         }
     }
 
     @Override
     public void teardown(ConfigurableComponent component) {
-        try (NarCloseable narCloseable = NarCloseable.withComponentNarLoader(component.getClass(), component.getIdentifier())) {
+        try (NarCloseable narCloseable = NarCloseable.withComponentNarLoader(extensionManager, 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());
+            extensionManager.removeInstanceClassLoader(component.getIdentifier());
         }
     }
 }


[7/7] nifi git commit: NIFI-5673 Set up property/assembly for new auto-load directory - Set up NarAutoLoader to watch directory for new files - Move NarAutoLoader to JettyServer since it will need access to ExtensionManager - Created NarLoader to shared

Posted by ma...@apache.org.
NIFI-5673 Set up property/assembly for new auto-load directory
- Set up NarAutoLoader to watch directory for new files
- Move NarAutoLoader to JettyServer since it will need access to ExtensionManager
- Created NarLoader to shared between NarAutoLoader and the framework
- Created nifi-framework-nar-loading-utils so we can use nifi-documentation to call DocGenerator
- Add additional bundles to overall map in NarClassLoaders as they are loaded
- Added handling of skipped NARs to include them in next iteration
- Added check of last modified timestamp on NARs
- Refactored JettyServer so we can load additional web contexts while the application is running
- Setting up unit tests
- Remove static use of ExtensionManager
- Adding unit tests for NarLoader
- Extracting interface for ExtensionManager and splitting discovery into it's own interface

This closes #3119.

Signed-off-by: Mark Payne <ma...@hotmail.com>


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

Branch: refs/heads/master
Commit: fdd8cdbb31a8977967d4437ba9a09104bf6cf450
Parents: 89295e5
Author: Bryan Bende <bb...@apache.org>
Authored: Fri Oct 12 11:15:30 2018 -0400
Committer: Mark Payne <ma...@hotmail.com>
Committed: Thu Nov 1 14:41:10 2018 -0400

----------------------------------------------------------------------
 nifi-assembly/src/main/assembly/common.xml      |   9 +
 .../org/apache/nifi/util/NiFiProperties.java    |   9 +-
 .../authorization/AuthorizerFactoryBean.java    |  11 +-
 .../main/resources/nifi-authorizer-context.xml  |   1 +
 .../apache/nifi/documentation/DocGenerator.java |  32 +-
 .../html/HtmlDocumentationWriter.java           |  12 +-
 .../html/HtmlProcessorDocumentationWriter.java  |   5 +
 .../nifi/documentation/DocGeneratorTest.java    |  29 +-
 .../html/HtmlDocumentationWriterTest.java       |  22 +-
 .../html/ProcessorDocumentationWriterTest.java  |  26 +-
 .../PopularVoteFlowElectionFactoryBean.java     |   8 +-
 .../node/NodeClusterCoordinator.java            |   5 +-
 .../NodeClusterCoordinatorFactoryBean.java      |   9 +-
 .../resources/nifi-cluster-manager-context.xml  |   2 +
 .../flow/TestPopularVoteFlowElection.java       |   8 +-
 .../nifi/cluster/integration/Cluster.java       |  22 +-
 .../apache/nifi/cluster/integration/Node.java   |  15 +-
 .../nifi/controller/AbstractComponentNode.java  |  38 +-
 .../apache/nifi/controller/ProcessorNode.java   |   7 +-
 .../reporting/ReportingTaskProvider.java        |   6 +
 .../service/ControllerServiceProvider.java      |   7 +
 .../controller/TestAbstractComponentNode.java   |   3 +-
 .../apache/nifi/controller/FlowController.java  | 136 ++---
 .../nifi/controller/StandardFlowService.java    |   6 +-
 .../controller/StandardFlowSynchronizer.java    |  15 +-
 .../nifi/controller/StandardProcessorNode.java  |  31 +-
 .../reporting/AbstractReportingTaskNode.java    |  11 +-
 .../reporting/StandardReportingTaskNode.java    |  11 +-
 .../scheduling/EventDrivenSchedulingAgent.java  |  14 +-
 .../scheduling/QuartzSchedulingAgent.java       |   2 +-
 .../scheduling/StandardProcessScheduler.java    |   6 +-
 .../scheduling/TimerDrivenSchedulingAgent.java  |   2 +-
 .../service/ControllerServiceLoader.java        |   2 +-
 ...ndardControllerServiceInvocationHandler.java |  11 +-
 .../service/StandardControllerServiceNode.java  |  17 +-
 .../StandardControllerServiceProvider.java      |  22 +-
 .../manager/StandardStateManagerProvider.java   |  26 +-
 .../nifi/controller/tasks/ConnectableTask.java  |   2 +-
 .../controller/tasks/ReportingTaskWrapper.java  |   9 +-
 .../nifi/fingerprint/FingerprintFactory.java    |  15 +-
 .../nifi/groups/StandardProcessGroup.java       |  19 +-
 .../StandardXMLFlowConfigurationDAO.java        |   8 +-
 .../flow/mapping/NiFiRegistryFlowMapper.java    |  11 +-
 .../spring/ExtensionManagerFactoryBean.java     |  45 ++
 .../nifi/spring/FlowControllerFactoryBean.java  |  12 +-
 .../java/org/apache/nifi/util/BundleUtils.java  |  33 +-
 .../src/main/resources/nifi-context.xml         |   5 +
 .../controller/StandardFlowServiceTest.java     |   6 +-
 .../nifi/controller/TestFlowController.java     | 109 ++--
 .../controller/TestStandardProcessorNode.java   |  81 +--
 .../reporting/TestStandardReportingContext.java |  27 +-
 .../scheduling/ProcessorLifecycleIT.java        |   8 +-
 .../scheduling/StandardProcessSchedulerIT.java  |  20 +-
 .../TestStandardProcessScheduler.java           |  22 +-
 .../StandardFlowSerializerTest.java             |  30 +-
 .../StandardControllerServiceProviderIT.java    |  28 +-
 .../StandardControllerServiceProviderTest.java  |  19 +-
 .../TestStandardControllerServiceProvider.java  |  45 +-
 .../fingerprint/FingerprintFactoryTest.java     |   8 +-
 .../nifi-framework-nar-loading-utils/pom.xml    |  46 ++
 .../org/apache/nifi/nar/ExtensionUiLoader.java  |  33 ++
 .../java/org/apache/nifi/nar/NarAutoLoader.java |  87 +++
 .../org/apache/nifi/nar/NarAutoLoaderTask.java  | 178 +++++++
 .../java/org/apache/nifi/nar/NarLoader.java     |  35 ++
 .../org/apache/nifi/nar/StandardNarLoader.java  | 167 ++++++
 .../java/org/apache/nifi/nar/TestNarLoader.java | 189 +++++++
 .../src/test/resources/README                   |   9 +
 .../src/test/resources/conf/nifi.properties     | 124 +++++
 .../nifi-example-processors-nar-1.0.nar         | Bin 0 -> 7221 bytes
 .../nifi-example-service-api-nar-1.0.nar        | Bin 0 -> 4306 bytes
 .../extensions/nifi-example-service-nar-1.1.nar | Bin 0 -> 6409 bytes
 .../test/resources/lib/nifi-framework-nar.nar   | Bin 0 -> 577 bytes
 .../test/resources/lib/nifi-jetty-bundle.nar    | Bin 0 -> 578 bytes
 .../test/resources/nifi-example-nars-source.zip | Bin 0 -> 39371 bytes
 ...ConfigurableComponentInitializerFactory.java |  10 +-
 .../nifi/init/ControllerServiceInitializer.java |  12 +-
 .../apache/nifi/init/ProcessorInitializer.java  |  12 +-
 .../nifi/init/ReportingTaskingInitializer.java  |  12 +-
 .../nifi/nar/ExtensionDiscoveringManager.java   |  47 ++
 .../org/apache/nifi/nar/ExtensionManager.java   | 504 ++----------------
 .../apache/nifi/nar/ExtensionManagerHolder.java |  61 +++
 .../java/org/apache/nifi/nar/NarCloseable.java  |   6 +-
 .../nifi/nar/NarThreadContextClassLoader.java   |   4 +-
 .../StandardExtensionDiscoveringManager.java    | 527 +++++++++++++++++++
 .../nar/NarThreadContextClassLoaderTest.java    |  29 +-
 .../org/apache/nifi/nar/NarClassLoaders.java    | 208 +++++++-
 .../apache/nifi/nar/NarClassLoadersHolder.java  |  41 ++
 .../java/org/apache/nifi/nar/NarLoadResult.java |  46 ++
 .../java/org/apache/nifi/nar/NarUnpacker.java   |   7 +-
 .../nifi-framework/nifi-resources/pom.xml       |   1 +
 .../src/main/resources/conf/nifi.properties     |   1 +
 .../src/main/java/org/apache/nifi/NiFi.java     |   3 +-
 .../nifi-framework/nifi-web/nifi-jetty/pom.xml  |   6 +-
 .../org/apache/nifi/web/server/JettyServer.java | 400 +++++++++-----
 .../nifi/ui/extension/UiExtensionMapping.java   |  14 +-
 .../StandardAuthorizableLookup.java             |  56 +-
 .../nifi/registry/flow/FlowRegistryUtils.java   |   6 +-
 .../org/apache/nifi/web/NiFiServiceFacade.java  |  31 ++
 .../nifi/web/StandardNiFiServiceFacade.java     |  24 +-
 .../org/apache/nifi/web/api/FlowResource.java   |   4 +-
 .../nifi/web/api/ProcessGroupResource.java      |  10 +-
 .../apache/nifi/web/api/VersionsResource.java   |   9 +-
 .../org/apache/nifi/web/api/dto/DtoFactory.java |  29 +-
 .../nifi/web/controller/ControllerFacade.java   |  20 +-
 .../web/controller/ControllerSearchService.java |   2 +-
 .../apache/nifi/web/dao/impl/ComponentDAO.java  |   4 +-
 .../dao/impl/StandardControllerServiceDAO.java  |  14 +-
 .../web/dao/impl/StandardProcessGroupDAO.java   |   2 +-
 .../nifi/web/dao/impl/StandardProcessorDAO.java |  12 +-
 .../web/dao/impl/StandardReportingTaskDAO.java  |  14 +-
 .../src/main/resources/nifi-web-api-context.xml |   1 +
 .../StandardAuthorizableLookupTest.java         |   8 +
 .../accesscontrol/AccessControlHelper.java      |  28 +-
 .../accesscontrol/ITAccessTokenEndpoint.java    |  28 +-
 .../test/resources/lib/nifi-framework-nar.nar   | Bin 0 -> 406 bytes
 .../test/resources/lib/nifi-jetty-bundle.nar    | Bin 0 -> 578 bytes
 .../LoginIdentityProviderFactoryBean.java       |   8 +-
 .../resources/nifi-web-security-context.xml     |   1 +
 .../nifi-framework/pom.xml                      |   3 +-
 nifi-nar-bundles/nifi-framework-bundle/pom.xml  |   5 +
 nifi-nar-bundles/nifi-jetty-bundle/pom.xml      |   5 +
 pom.xml                                         |   6 +
 122 files changed, 3147 insertions(+), 1162 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/nifi/blob/fdd8cdbb/nifi-assembly/src/main/assembly/common.xml
----------------------------------------------------------------------
diff --git a/nifi-assembly/src/main/assembly/common.xml b/nifi-assembly/src/main/assembly/common.xml
index e4a8d7d..cad953d 100644
--- a/nifi-assembly/src/main/assembly/common.xml
+++ b/nifi-assembly/src/main/assembly/common.xml
@@ -91,6 +91,15 @@
             </unpackOptions>
         </dependencySet>
     </dependencySets>
+    <fileSets>
+        <fileSet>
+            <directory>.</directory>
+            <outputDirectory>extensions</outputDirectory>
+            <excludes>
+                <exclude>*/**</exclude>
+            </excludes>
+        </fileSet>
+    </fileSets>
     <files>
         <file>
             <source>./README.md</source>

http://git-wip-us.apache.org/repos/asf/nifi/blob/fdd8cdbb/nifi-commons/nifi-properties/src/main/java/org/apache/nifi/util/NiFiProperties.java
----------------------------------------------------------------------
diff --git a/nifi-commons/nifi-properties/src/main/java/org/apache/nifi/util/NiFiProperties.java b/nifi-commons/nifi-properties/src/main/java/org/apache/nifi/util/NiFiProperties.java
index 562f8c0..9132eae 100644
--- a/nifi-commons/nifi-properties/src/main/java/org/apache/nifi/util/NiFiProperties.java
+++ b/nifi-commons/nifi-properties/src/main/java/org/apache/nifi/util/NiFiProperties.java
@@ -62,6 +62,7 @@ public abstract class NiFiProperties {
     public static final String FLOW_CONTROLLER_GRACEFUL_SHUTDOWN_PERIOD = "nifi.flowcontroller.graceful.shutdown.period";
     public static final String NAR_LIBRARY_DIRECTORY = "nifi.nar.library.directory";
     public static final String NAR_LIBRARY_DIRECTORY_PREFIX = "nifi.nar.library.directory.";
+    public static final String NAR_LIBRARY_AUTOLOAD_DIRECTORY = "nifi.nar.library.autoload.directory";
     public static final String NAR_WORKING_DIRECTORY = "nifi.nar.working.directory";
     public static final String COMPONENT_DOCS_DIRECTORY = "nifi.documentation.working.directory";
     public static final String SENSITIVE_PROPS_KEY = "nifi.sensitive.props.key";
@@ -247,6 +248,7 @@ public abstract class NiFiProperties {
     public static final String DEFAULT_NAR_WORKING_DIR = "./work/nar";
     public static final String DEFAULT_COMPONENT_DOCS_DIRECTORY = "./work/docs/components";
     public static final String DEFAULT_NAR_LIBRARY_DIR = "./lib";
+    public static final String DEFAULT_NAR_LIBRARY_AUTOLOAD_DIR = "./extensions";
     public static final String DEFAULT_FLOWFILE_REPO_PARTITIONS = "256";
     public static final String DEFAULT_FLOWFILE_CHECKPOINT_INTERVAL = "2 min";
     public static final int DEFAULT_MAX_FLOWFILES_PER_CLAIM = 100;
@@ -650,7 +652,8 @@ public abstract class NiFiProperties {
         for (String propertyName : getPropertyKeys()) {
             // determine if the property is a nar library path
             if (StringUtils.startsWith(propertyName, NAR_LIBRARY_DIRECTORY_PREFIX)
-                    || NAR_LIBRARY_DIRECTORY.equals(propertyName)) {
+                    || NAR_LIBRARY_DIRECTORY.equals(propertyName)
+                    || NAR_LIBRARY_AUTOLOAD_DIRECTORY.equals(propertyName)) {
                 // attempt to resolve the path specified
                 String narLib = getProperty(propertyName);
                 if (!StringUtils.isBlank(narLib)) {
@@ -666,6 +669,10 @@ public abstract class NiFiProperties {
         return narLibraryPaths;
     }
 
+    public File getNarAutoLoadDirectory() {
+        return new File(getProperty(NAR_LIBRARY_AUTOLOAD_DIRECTORY, DEFAULT_NAR_LIBRARY_AUTOLOAD_DIR));
+    }
+
     // getters for ui properties //
 
     /**

http://git-wip-us.apache.org/repos/asf/nifi/blob/fdd8cdbb/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-authorizer/src/main/java/org/apache/nifi/authorization/AuthorizerFactoryBean.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-authorizer/src/main/java/org/apache/nifi/authorization/AuthorizerFactoryBean.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-authorizer/src/main/java/org/apache/nifi/authorization/AuthorizerFactoryBean.java
index 79d3757..1e12264 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-authorizer/src/main/java/org/apache/nifi/authorization/AuthorizerFactoryBean.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-authorizer/src/main/java/org/apache/nifi/authorization/AuthorizerFactoryBean.java
@@ -89,6 +89,7 @@ public class AuthorizerFactoryBean implements FactoryBean, DisposableBean, UserG
 
     private Authorizer authorizer;
     private NiFiProperties properties;
+    private ExtensionManager extensionManager;
     private final Map<String, UserGroupProvider> userGroupProviders = new HashMap<>();
     private final Map<String, AccessPolicyProvider> accessPolicyProviders = new HashMap<>();
     private final Map<String, Authorizer> authorizers = new HashMap<>();
@@ -208,7 +209,7 @@ public class AuthorizerFactoryBean implements FactoryBean, DisposableBean, UserG
 
     private UserGroupProvider createUserGroupProvider(final String identifier, final String userGroupProviderClassName) throws Exception {
         // get the classloader for the specified user group provider
-        final List<Bundle> userGroupProviderBundles = ExtensionManager.getBundles(userGroupProviderClassName);
+        final List<Bundle> userGroupProviderBundles = extensionManager.getBundles(userGroupProviderClassName);
 
         if (userGroupProviderBundles.size() == 0) {
             throw new Exception(String.format("The specified user group provider class '%s' is not known to this nifi.", userGroupProviderClassName));
@@ -256,7 +257,7 @@ public class AuthorizerFactoryBean implements FactoryBean, DisposableBean, UserG
 
     private AccessPolicyProvider createAccessPolicyProvider(final String identifier, final String accessPolicyProviderClassName) throws Exception {
         // get the classloader for the specified access policy provider
-        final List<Bundle> accessPolicyProviderBundles = ExtensionManager.getBundles(accessPolicyProviderClassName);
+        final List<Bundle> accessPolicyProviderBundles = extensionManager.getBundles(accessPolicyProviderClassName);
 
         if (accessPolicyProviderBundles.size() == 0) {
             throw new Exception(String.format("The specified access policy provider class '%s' is not known to this nifi.", accessPolicyProviderClassName));
@@ -304,7 +305,7 @@ public class AuthorizerFactoryBean implements FactoryBean, DisposableBean, UserG
 
     private Authorizer createAuthorizer(final String identifier, final String authorizerClassName, final String classpathResources) throws Exception {
         // get the classloader for the specified authorizer
-        final List<Bundle> authorizerBundles = ExtensionManager.getBundles(authorizerClassName);
+        final List<Bundle> authorizerBundles = extensionManager.getBundles(authorizerClassName);
 
         if (authorizerBundles.size() == 0) {
             throw new Exception(String.format("The specified authorizer class '%s' is not known to this nifi.", authorizerClassName));
@@ -537,4 +538,8 @@ public class AuthorizerFactoryBean implements FactoryBean, DisposableBean, UserG
         this.properties = properties;
     }
 
+    public void setExtensionManager(ExtensionManager extensionManager) {
+        this.extensionManager = extensionManager;
+    }
+
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/fdd8cdbb/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-authorizer/src/main/resources/nifi-authorizer-context.xml
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-authorizer/src/main/resources/nifi-authorizer-context.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-authorizer/src/main/resources/nifi-authorizer-context.xml
index 71bf684..6c484fc 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-authorizer/src/main/resources/nifi-authorizer-context.xml
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-authorizer/src/main/resources/nifi-authorizer-context.xml
@@ -21,6 +21,7 @@
     <!-- user/entity authorizer -->
     <bean id="authorizer" class="org.apache.nifi.authorization.AuthorizerFactoryBean">
         <property name="properties" ref="nifiProperties"/>
+        <property name="extensionManager" ref="extensionManager" />
     </bean>
 
 </beans>

http://git-wip-us.apache.org/repos/asf/nifi/blob/fdd8cdbb/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 dd99927..4a01b35 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
@@ -56,14 +56,14 @@ public class DocGenerator {
      * @param properties to lookup nifi properties
      * @param extensionMapping extension mapping
      */
-    public static void generate(final NiFiProperties properties, final ExtensionMapping extensionMapping) {
+    public static void generate(final NiFiProperties properties, final ExtensionManager extensionManager, final ExtensionMapping extensionMapping) {
         final File explodedNiFiDocsDir = properties.getComponentDocumentationWorkingDirectory();
 
         logger.debug("Generating documentation for: " + extensionMapping.size() + " components in: " + explodedNiFiDocsDir);
 
-        documentConfigurableComponent(ExtensionManager.getExtensions(Processor.class), explodedNiFiDocsDir);
-        documentConfigurableComponent(ExtensionManager.getExtensions(ControllerService.class), explodedNiFiDocsDir);
-        documentConfigurableComponent(ExtensionManager.getExtensions(ReportingTask.class), explodedNiFiDocsDir);
+        documentConfigurableComponent(extensionManager.getExtensions(Processor.class), explodedNiFiDocsDir, extensionManager);
+        documentConfigurableComponent(extensionManager.getExtensions(ControllerService.class), explodedNiFiDocsDir, extensionManager);
+        documentConfigurableComponent(extensionManager.getExtensions(ReportingTask.class), explodedNiFiDocsDir, extensionManager);
     }
 
     /**
@@ -72,12 +72,12 @@ public class DocGenerator {
      * @param extensionClasses types of a configurable component
      * @param explodedNiFiDocsDir base directory of component documentation
      */
-    private static void documentConfigurableComponent(final Set<Class> extensionClasses, final File explodedNiFiDocsDir) {
+    public static void documentConfigurableComponent(final Set<Class> extensionClasses, final File explodedNiFiDocsDir, final ExtensionManager extensionManager) {
         for (final Class<?> extensionClass : extensionClasses) {
             if (ConfigurableComponent.class.isAssignableFrom(extensionClass)) {
                 final String extensionClassName = extensionClass.getCanonicalName();
 
-                final Bundle bundle = ExtensionManager.getBundle(extensionClass.getClassLoader());
+                final Bundle bundle = extensionManager.getBundle(extensionClass.getClassLoader());
                 if (bundle == null) {
                     logger.warn("No coordinate found for {}, skipping...", new Object[] {extensionClassName});
                     continue;
@@ -91,7 +91,7 @@ public class DocGenerator {
                 final Class<? extends ConfigurableComponent> componentClass = extensionClass.asSubclass(ConfigurableComponent.class);
                 try {
                     logger.debug("Documenting: " + componentClass);
-                    document(componentDirectory, componentClass, coordinate);
+                    document(extensionManager, componentDirectory, componentClass, coordinate);
                 } catch (Exception e) {
                     logger.warn("Unable to document: " + componentClass, e);
                 }
@@ -111,14 +111,17 @@ public class DocGenerator {
      * @throws IOException ioe
      * @throws InitializationException ie
      */
-    private static void document(final File componentDocsDir, final Class<? extends ConfigurableComponent> componentClass, final BundleCoordinate bundleCoordinate)
+    private static void document(final ExtensionManager extensionManager,
+                                 final File componentDocsDir,
+                                 final Class<? extends ConfigurableComponent> componentClass,
+                                 final BundleCoordinate bundleCoordinate)
             throws InstantiationException, IllegalAccessException, IOException, InitializationException {
 
         // 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 ConfigurableComponent component = extensionManager.getTempComponent(classType, bundleCoordinate);
 
-        final DocumentationWriter writer = getDocumentWriter(componentClass);
+        final DocumentationWriter writer = getDocumentWriter(extensionManager, componentClass);
 
         final File baseDocumentationFile = new File(componentDocsDir, "index.html");
         if (baseDocumentationFile.exists()) {
@@ -138,13 +141,14 @@ public class DocGenerator {
      * @return a DocumentationWriter capable of generating documentation for
      * that specific type of class
      */
-    private static DocumentationWriter getDocumentWriter(final Class<? extends ConfigurableComponent> componentClass) {
+    private static DocumentationWriter getDocumentWriter(final ExtensionManager extensionManager,
+                                                         final Class<? extends ConfigurableComponent> componentClass) {
         if (Processor.class.isAssignableFrom(componentClass)) {
-            return new HtmlProcessorDocumentationWriter();
+            return new HtmlProcessorDocumentationWriter(extensionManager);
         } else if (ControllerService.class.isAssignableFrom(componentClass)) {
-            return new HtmlDocumentationWriter();
+            return new HtmlDocumentationWriter(extensionManager);
         } else if (ReportingTask.class.isAssignableFrom(componentClass)) {
-            return new HtmlDocumentationWriter();
+            return new HtmlDocumentationWriter(extensionManager);
         }
 
         return null;

http://git-wip-us.apache.org/repos/asf/nifi/blob/fdd8cdbb/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/html/HtmlDocumentationWriter.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/html/HtmlDocumentationWriter.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/html/HtmlDocumentationWriter.java
index fb2c5a0..1b401f9 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/html/HtmlDocumentationWriter.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/html/HtmlDocumentationWriter.java
@@ -67,6 +67,12 @@ public class HtmlDocumentationWriter implements DocumentationWriter {
      */
     public static final String ADDITIONAL_DETAILS_HTML = "additionalDetails.html";
 
+    private final ExtensionManager extensionManager;
+
+    public HtmlDocumentationWriter(final ExtensionManager extensionManager) {
+        this.extensionManager = extensionManager;
+    }
+
     @Override
     public void write(final ConfigurableComponent configurableComponent, final OutputStream streamToWriteTo,
             final boolean includesAdditionalDocumentation) throws IOException {
@@ -848,7 +854,7 @@ public class HtmlDocumentationWriter implements DocumentationWriter {
         final List<Class<? extends ControllerService>> implementations = new ArrayList<>();
 
         // first get all ControllerService implementations
-        final Set<Class> controllerServices = ExtensionManager.getExtensions(ControllerService.class);
+        final Set<Class> controllerServices = extensionManager.getExtensions(ControllerService.class);
 
         // then iterate over all controller services looking for any that is a child of the parent
         // ControllerService API that was passed in as a parameter
@@ -891,7 +897,7 @@ public class HtmlDocumentationWriter implements DocumentationWriter {
         int index = 0;
         for (final Class<? extends ConfigurableComponent> linkedComponent : linkedComponents ) {
             final String linkedComponentName = linkedComponent.getName();
-            final List<Bundle> linkedComponentBundles = ExtensionManager.getBundles(linkedComponentName);
+            final List<Bundle> linkedComponentBundles = extensionManager.getBundles(linkedComponentName);
             if (linkedComponentBundles != null && linkedComponentBundles.size() > 0) {
                 final Bundle firstLinkedComponentBundle = linkedComponentBundles.get(0);
                 final BundleCoordinate coordinate = firstLinkedComponentBundle.getBundleDetails().getCoordinate();
@@ -927,7 +933,7 @@ public class HtmlDocumentationWriter implements DocumentationWriter {
                     }
                 }
 
-                final List<Bundle> linkedComponentBundles = ExtensionManager.getBundles(className);
+                final List<Bundle> linkedComponentBundles = extensionManager.getBundles(className);
 
                 if (linkedComponentBundles != null && linkedComponentBundles.size() > 0) {
                     final Bundle firstBundle = linkedComponentBundles.get(0);

http://git-wip-us.apache.org/repos/asf/nifi/blob/fdd8cdbb/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/html/HtmlProcessorDocumentationWriter.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/html/HtmlProcessorDocumentationWriter.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/html/HtmlProcessorDocumentationWriter.java
index 6ac34a2..5192222 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/html/HtmlProcessorDocumentationWriter.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/html/HtmlProcessorDocumentationWriter.java
@@ -29,6 +29,7 @@ import org.apache.nifi.annotation.behavior.ReadsAttributes;
 import org.apache.nifi.annotation.behavior.WritesAttribute;
 import org.apache.nifi.annotation.behavior.WritesAttributes;
 import org.apache.nifi.components.ConfigurableComponent;
+import org.apache.nifi.nar.ExtensionManager;
 import org.apache.nifi.processor.Processor;
 import org.apache.nifi.processor.Relationship;
 
@@ -40,6 +41,10 @@ import org.apache.nifi.processor.Relationship;
  */
 public class HtmlProcessorDocumentationWriter extends HtmlDocumentationWriter {
 
+    public HtmlProcessorDocumentationWriter(ExtensionManager extensionManager) {
+        super(extensionManager);
+    }
+
     @Override
     protected void writeAdditionalBodyInfo(final ConfigurableComponent configurableComponent,
             final XMLStreamWriter xmlStreamWriter) throws XMLStreamException {

http://git-wip-us.apache.org/repos/asf/nifi/blob/fdd8cdbb/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 e3b3ffd..3e93427 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
@@ -16,27 +16,29 @@
  */
 package org.apache.nifi.documentation;
 
-import java.io.BufferedInputStream;
-import java.io.File;
-import java.io.FileInputStream;
-import java.io.IOException;
-import java.io.InputStream;
-import java.nio.charset.Charset;
-import java.util.Properties;
-import java.util.Set;
 import org.apache.commons.io.FileUtils;
 import org.apache.nifi.bundle.Bundle;
 import org.apache.nifi.bundle.BundleCoordinate;
-import org.apache.nifi.nar.ExtensionManager;
+import org.apache.nifi.nar.ExtensionDiscoveringManager;
 import org.apache.nifi.nar.ExtensionMapping;
-import org.apache.nifi.nar.NarClassLoaders;
+import org.apache.nifi.nar.NarClassLoadersHolder;
 import org.apache.nifi.nar.NarUnpacker;
+import org.apache.nifi.nar.StandardExtensionDiscoveringManager;
 import org.apache.nifi.nar.SystemBundle;
 import org.apache.nifi.util.NiFiProperties;
 import org.junit.Assert;
 import org.junit.Test;
 import org.junit.rules.TemporaryFolder;
 
+import java.io.BufferedInputStream;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.charset.Charset;
+import java.util.Properties;
+import java.util.Set;
+
 public class DocGeneratorTest {
 
     @Test
@@ -51,11 +53,12 @@ public class DocGeneratorTest {
         final Bundle systemBundle = SystemBundle.create(properties);
         final ExtensionMapping mapping = NarUnpacker.unpackNars(properties, systemBundle);
 
-        NarClassLoaders.getInstance().init(properties.getFrameworkWorkingDirectory(), properties.getExtensionsWorkingDirectory());
+        NarClassLoadersHolder.getInstance().init(properties.getFrameworkWorkingDirectory(), properties.getExtensionsWorkingDirectory());
 
-        ExtensionManager.discoverExtensions(systemBundle, NarClassLoaders.getInstance().getBundles());
+        final ExtensionDiscoveringManager extensionManager = new StandardExtensionDiscoveringManager();
+        extensionManager.discoverExtensions(systemBundle, NarClassLoadersHolder.getInstance().getBundles());
 
-        DocGenerator.generate(properties, mapping);
+        DocGenerator.generate(properties, extensionManager, mapping);
 
         final String extensionClassName = "org.apache.nifi.processors.WriteResourceToStream";
         final BundleCoordinate coordinate = mapping.getProcessorNames().get(extensionClassName).stream().findFirst().get();

http://git-wip-us.apache.org/repos/asf/nifi/blob/fdd8cdbb/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/java/org/apache/nifi/documentation/html/HtmlDocumentationWriterTest.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/java/org/apache/nifi/documentation/html/HtmlDocumentationWriterTest.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/java/org/apache/nifi/documentation/html/HtmlDocumentationWriterTest.java
index 84b8b1f..585fe1e 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/java/org/apache/nifi/documentation/html/HtmlDocumentationWriterTest.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/java/org/apache/nifi/documentation/html/HtmlDocumentationWriterTest.java
@@ -28,9 +28,12 @@ import org.apache.nifi.init.ControllerServiceInitializer;
 import org.apache.nifi.init.ReportingTaskingInitializer;
 import org.apache.nifi.mock.MockControllerServiceInitializationContext;
 import org.apache.nifi.mock.MockReportingInitializationContext;
+import org.apache.nifi.nar.ExtensionManager;
+import org.apache.nifi.nar.StandardExtensionDiscoveringManager;
 import org.apache.nifi.reporting.InitializationException;
 import org.apache.nifi.reporting.ReportingTask;
 import org.junit.Assert;
+import org.junit.Before;
 import org.junit.Test;
 
 import java.io.ByteArrayOutputStream;
@@ -41,6 +44,13 @@ import static org.junit.Assert.assertEquals;
 
 public class HtmlDocumentationWriterTest {
 
+    private ExtensionManager extensionManager;
+
+    @Before
+    public void setup() {
+        extensionManager = new StandardExtensionDiscoveringManager();
+    }
+
     @Test
     public void testJoin() {
         assertEquals("a, b, c", HtmlDocumentationWriter.join(new String[] { "a", "b", "c" }, ", "));
@@ -52,10 +62,10 @@ public class HtmlDocumentationWriterTest {
     public void testDocumentControllerService() throws InitializationException, IOException {
 
         FullyDocumentedControllerService controllerService = new FullyDocumentedControllerService();
-        ControllerServiceInitializer initializer = new ControllerServiceInitializer();
+        ControllerServiceInitializer initializer = new ControllerServiceInitializer(extensionManager);
         initializer.initialize(controllerService);
 
-        DocumentationWriter writer = new HtmlDocumentationWriter();
+        DocumentationWriter writer = new HtmlDocumentationWriter(extensionManager);
 
         ByteArrayOutputStream baos = new ByteArrayOutputStream();
 
@@ -102,10 +112,10 @@ public class HtmlDocumentationWriterTest {
     public void testDocumentReportingTask() throws InitializationException, IOException {
 
         FullyDocumentedReportingTask reportingTask = new FullyDocumentedReportingTask();
-        ReportingTaskingInitializer initializer = new ReportingTaskingInitializer();
+        ReportingTaskingInitializer initializer = new ReportingTaskingInitializer(extensionManager);
         initializer.initialize(reportingTask);
 
-        DocumentationWriter writer = new HtmlDocumentationWriter();
+        DocumentationWriter writer = new HtmlDocumentationWriter(extensionManager);
 
         ByteArrayOutputStream baos = new ByteArrayOutputStream();
 
@@ -152,7 +162,7 @@ public class HtmlDocumentationWriterTest {
         ControllerService controllerService = new ControllerServiceWithLogger();
         controllerService.initialize(new MockControllerServiceInitializationContext());
 
-        DocumentationWriter writer = new HtmlDocumentationWriter();
+        DocumentationWriter writer = new HtmlDocumentationWriter(extensionManager);
 
         ByteArrayOutputStream baos = new ByteArrayOutputStream();
 
@@ -168,7 +178,7 @@ public class HtmlDocumentationWriterTest {
         ReportingTask controllerService = new ReportingTaskWithLogger();
         controllerService.initialize(new MockReportingInitializationContext());
 
-        DocumentationWriter writer = new HtmlDocumentationWriter();
+        DocumentationWriter writer = new HtmlDocumentationWriter(extensionManager);
 
         ByteArrayOutputStream baos = new ByteArrayOutputStream();
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/fdd8cdbb/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/java/org/apache/nifi/documentation/html/ProcessorDocumentationWriterTest.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/java/org/apache/nifi/documentation/html/ProcessorDocumentationWriterTest.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/java/org/apache/nifi/documentation/html/ProcessorDocumentationWriterTest.java
index 4e45b7b..06c8e83 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/java/org/apache/nifi/documentation/html/ProcessorDocumentationWriterTest.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/java/org/apache/nifi/documentation/html/ProcessorDocumentationWriterTest.java
@@ -32,6 +32,8 @@ import org.apache.nifi.documentation.example.FullyDocumentedProcessor;
 import org.apache.nifi.documentation.example.NakedProcessor;
 import org.apache.nifi.documentation.example.ProcessorWithLogger;
 import org.apache.nifi.init.ProcessorInitializer;
+import org.apache.nifi.nar.ExtensionManager;
+import org.apache.nifi.nar.StandardExtensionDiscoveringManager;
 import org.junit.Assert;
 import org.junit.Test;
 
@@ -39,11 +41,13 @@ public class ProcessorDocumentationWriterTest {
 
     @Test
     public void testFullyDocumentedProcessor() throws IOException {
+        ExtensionManager extensionManager = new StandardExtensionDiscoveringManager();
+
         FullyDocumentedProcessor processor = new FullyDocumentedProcessor();
-        ProcessorInitializer initializer = new ProcessorInitializer();
+        ProcessorInitializer initializer = new ProcessorInitializer(extensionManager);
         initializer.initialize(processor);
 
-        DocumentationWriter writer = new HtmlProcessorDocumentationWriter();
+        DocumentationWriter writer = new HtmlProcessorDocumentationWriter(extensionManager);
 
         ByteArrayOutputStream baos = new ByteArrayOutputStream();
 
@@ -114,11 +118,13 @@ public class ProcessorDocumentationWriterTest {
 
     @Test
     public void testNakedProcessor() throws IOException {
+        ExtensionManager extensionManager = new StandardExtensionDiscoveringManager();
+
         NakedProcessor processor = new NakedProcessor();
-        ProcessorInitializer initializer = new ProcessorInitializer();
+        ProcessorInitializer initializer = new ProcessorInitializer(extensionManager);
         initializer.initialize(processor);
 
-        DocumentationWriter writer = new HtmlProcessorDocumentationWriter();
+        DocumentationWriter writer = new HtmlProcessorDocumentationWriter(extensionManager);
 
         ByteArrayOutputStream baos = new ByteArrayOutputStream();
 
@@ -152,11 +158,13 @@ public class ProcessorDocumentationWriterTest {
 
     @Test
     public void testProcessorWithLoggerInitialization() throws IOException {
+        ExtensionManager extensionManager = new StandardExtensionDiscoveringManager();
+
         ProcessorWithLogger processor = new ProcessorWithLogger();
-        ProcessorInitializer initializer = new ProcessorInitializer();
+        ProcessorInitializer initializer = new ProcessorInitializer(extensionManager);
         initializer.initialize(processor);
 
-        DocumentationWriter writer = new HtmlProcessorDocumentationWriter();
+        DocumentationWriter writer = new HtmlProcessorDocumentationWriter(extensionManager);
 
         ByteArrayOutputStream baos = new ByteArrayOutputStream();
 
@@ -170,11 +178,13 @@ public class ProcessorDocumentationWriterTest {
 
     @Test
     public void testDeprecatedProcessor() throws IOException {
+        ExtensionManager extensionManager = new StandardExtensionDiscoveringManager();
+
         DeprecatedProcessor processor = new DeprecatedProcessor();
-        ProcessorInitializer initializer = new ProcessorInitializer();
+        ProcessorInitializer initializer = new ProcessorInitializer(extensionManager);
         initializer.initialize(processor);
 
-        DocumentationWriter writer = new HtmlProcessorDocumentationWriter();
+        DocumentationWriter writer = new HtmlProcessorDocumentationWriter(extensionManager);
 
         ByteArrayOutputStream baos = new ByteArrayOutputStream();
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/fdd8cdbb/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/flow/PopularVoteFlowElectionFactoryBean.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/flow/PopularVoteFlowElectionFactoryBean.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/flow/PopularVoteFlowElectionFactoryBean.java
index 4ea4225..9800d26 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/flow/PopularVoteFlowElectionFactoryBean.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/flow/PopularVoteFlowElectionFactoryBean.java
@@ -20,6 +20,7 @@ package org.apache.nifi.cluster.coordination.flow;
 import java.util.concurrent.TimeUnit;
 import org.apache.nifi.encrypt.StringEncryptor;
 import org.apache.nifi.fingerprint.FingerprintFactory;
+import org.apache.nifi.nar.ExtensionManager;
 import org.apache.nifi.util.FormatUtils;
 import org.apache.nifi.util.NiFiProperties;
 import org.slf4j.Logger;
@@ -29,6 +30,7 @@ import org.springframework.beans.factory.FactoryBean;
 public class PopularVoteFlowElectionFactoryBean implements FactoryBean<PopularVoteFlowElection> {
     private static final Logger logger = LoggerFactory.getLogger(PopularVoteFlowElectionFactoryBean.class);
     private NiFiProperties properties;
+    private ExtensionManager extensionManager;
 
     @Override
     public PopularVoteFlowElection getObject() {
@@ -47,7 +49,7 @@ public class PopularVoteFlowElectionFactoryBean implements FactoryBean<PopularVo
         final String provider =  properties.getProperty(NiFiProperties.SENSITIVE_PROPS_PROVIDER);
         final String password =  properties.getProperty(NiFiProperties.SENSITIVE_PROPS_KEY);
         final StringEncryptor encryptor = StringEncryptor.createEncryptor(algorithm, provider, password);
-        final FingerprintFactory fingerprintFactory = new FingerprintFactory(encryptor);
+        final FingerprintFactory fingerprintFactory = new FingerprintFactory(encryptor, extensionManager);
         return new PopularVoteFlowElection(maxWaitMillis, TimeUnit.MILLISECONDS, maxNodes, fingerprintFactory);
     }
 
@@ -64,4 +66,8 @@ public class PopularVoteFlowElectionFactoryBean implements FactoryBean<PopularVo
     public void setProperties(final NiFiProperties properties) {
         this.properties = properties;
     }
+
+    public void setExtensionManager(ExtensionManager extensionManager) {
+        this.extensionManager = extensionManager;
+    }
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/fdd8cdbb/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/node/NodeClusterCoordinator.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/node/NodeClusterCoordinator.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/node/NodeClusterCoordinator.java
index 8c83a1d..66eec26 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/node/NodeClusterCoordinator.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/node/NodeClusterCoordinator.java
@@ -64,6 +64,7 @@ import org.apache.nifi.components.state.StateMap;
 import org.apache.nifi.controller.leader.election.LeaderElectionManager;
 import org.apache.nifi.controller.state.manager.StandardStateManagerProvider;
 import org.apache.nifi.events.EventReporter;
+import org.apache.nifi.nar.ExtensionManager;
 import org.apache.nifi.registry.VariableRegistry;
 import org.apache.nifi.reporting.Severity;
 import org.apache.nifi.services.FlowService;
@@ -123,9 +124,9 @@ public class NodeClusterCoordinator implements ClusterCoordinator, ProtocolHandl
 
     public NodeClusterCoordinator(final ClusterCoordinationProtocolSenderListener senderListener, final EventReporter eventReporter, final LeaderElectionManager leaderElectionManager,
                                   final FlowElection flowElection, final ClusterNodeFirewall firewall, final RevisionManager revisionManager, final NiFiProperties nifiProperties,
-                                  final NodeProtocolSender nodeProtocolSender) throws IOException {
+                                  final ExtensionManager extensionManager, final NodeProtocolSender nodeProtocolSender) throws IOException {
         this(senderListener, eventReporter, leaderElectionManager, flowElection, firewall, revisionManager, nifiProperties, nodeProtocolSender,
-            StandardStateManagerProvider.create(nifiProperties, VariableRegistry.EMPTY_REGISTRY));
+            StandardStateManagerProvider.create(nifiProperties, VariableRegistry.EMPTY_REGISTRY, extensionManager));
     }
 
     public NodeClusterCoordinator(final ClusterCoordinationProtocolSenderListener senderListener, final EventReporter eventReporter, final LeaderElectionManager leaderElectionManager,

http://git-wip-us.apache.org/repos/asf/nifi/blob/fdd8cdbb/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/spring/NodeClusterCoordinatorFactoryBean.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/spring/NodeClusterCoordinatorFactoryBean.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/spring/NodeClusterCoordinatorFactoryBean.java
index ac79a42..4946d26 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/spring/NodeClusterCoordinatorFactoryBean.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/spring/NodeClusterCoordinatorFactoryBean.java
@@ -24,6 +24,7 @@ import org.apache.nifi.cluster.protocol.NodeProtocolSender;
 import org.apache.nifi.cluster.protocol.impl.ClusterCoordinationProtocolSenderListener;
 import org.apache.nifi.controller.leader.election.LeaderElectionManager;
 import org.apache.nifi.events.EventReporter;
+import org.apache.nifi.nar.ExtensionManager;
 import org.apache.nifi.util.NiFiProperties;
 import org.apache.nifi.web.revision.RevisionManager;
 import org.springframework.beans.BeansException;
@@ -34,6 +35,7 @@ import org.springframework.context.ApplicationContextAware;
 public class NodeClusterCoordinatorFactoryBean implements FactoryBean<NodeClusterCoordinator>, ApplicationContextAware {
     private ApplicationContext applicationContext;
     private NiFiProperties properties;
+    private ExtensionManager extensionManager;
 
     private NodeClusterCoordinator nodeClusterCoordinator = null;
 
@@ -48,7 +50,8 @@ public class NodeClusterCoordinatorFactoryBean implements FactoryBean<NodeCluste
             final LeaderElectionManager electionManager = applicationContext.getBean("leaderElectionManager", LeaderElectionManager.class);
             final FlowElection flowElection = applicationContext.getBean("flowElection", FlowElection.class);
             final NodeProtocolSender nodeProtocolSender = applicationContext.getBean("nodeProtocolSender", NodeProtocolSender.class);
-            nodeClusterCoordinator = new NodeClusterCoordinator(protocolSenderListener, eventReporter, electionManager, flowElection, clusterFirewall, revisionManager, properties, nodeProtocolSender);
+            nodeClusterCoordinator = new NodeClusterCoordinator(protocolSenderListener, eventReporter, electionManager, flowElection, clusterFirewall,
+                    revisionManager, properties, extensionManager, nodeProtocolSender);
         }
 
         return nodeClusterCoordinator;
@@ -73,4 +76,8 @@ public class NodeClusterCoordinatorFactoryBean implements FactoryBean<NodeCluste
         this.properties = properties;
     }
 
+    public void setExtensionManager(ExtensionManager extensionManager) {
+        this.extensionManager = extensionManager;
+    }
+
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/fdd8cdbb/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/resources/nifi-cluster-manager-context.xml
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/resources/nifi-cluster-manager-context.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/resources/nifi-cluster-manager-context.xml
index d261590..c1a7665 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/resources/nifi-cluster-manager-context.xml
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/resources/nifi-cluster-manager-context.xml
@@ -43,11 +43,13 @@
 
     <bean id="flowElection" class="org.apache.nifi.cluster.coordination.flow.PopularVoteFlowElectionFactoryBean">
         <property name="properties" ref="nifiProperties" />
+        <property name="extensionManager" ref="extensionManager" />
     </bean>
 
     <!-- Cluster Coordinator -->
     <bean id="clusterCoordinator" class="org.apache.nifi.cluster.spring.NodeClusterCoordinatorFactoryBean">
         <property name="properties" ref="nifiProperties"/>
+        <property name="extensionManager" ref="extensionManager" />
     </bean>
     
     <!-- Heartbeat Monitor -->

http://git-wip-us.apache.org/repos/asf/nifi/blob/fdd8cdbb/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/coordination/flow/TestPopularVoteFlowElection.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/coordination/flow/TestPopularVoteFlowElection.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/coordination/flow/TestPopularVoteFlowElection.java
index 9c833b5..240fe49 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/coordination/flow/TestPopularVoteFlowElection.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/coordination/flow/TestPopularVoteFlowElection.java
@@ -37,6 +37,8 @@ import org.apache.nifi.cluster.protocol.NodeIdentifier;
 import org.apache.nifi.cluster.protocol.StandardDataFlow;
 import org.apache.nifi.encrypt.StringEncryptor;
 import org.apache.nifi.fingerprint.FingerprintFactory;
+import org.apache.nifi.nar.ExtensionManager;
+import org.apache.nifi.nar.StandardExtensionDiscoveringManager;
 import org.apache.nifi.util.NiFiProperties;
 import org.junit.Test;
 import org.mockito.Mockito;
@@ -153,7 +155,8 @@ public class TestPopularVoteFlowElection {
 
     @Test
     public void testAutoGeneratedVsPopulatedFlowElection() throws IOException {
-        final FingerprintFactory fingerprintFactory = new FingerprintFactory(createEncryptorFromProperties(getNiFiProperties()));
+        final ExtensionManager extensionManager = new StandardExtensionDiscoveringManager();
+        final FingerprintFactory fingerprintFactory = new FingerprintFactory(createEncryptorFromProperties(getNiFiProperties()), extensionManager);
         final PopularVoteFlowElection election = new PopularVoteFlowElection(1, TimeUnit.MINUTES, 4, fingerprintFactory);
         final byte[] emptyFlow = Files.readAllBytes(Paths.get("src/test/resources/conf/auto-generated-empty-flow.xml"));
         final byte[] nonEmptyFlow = Files.readAllBytes(Paths.get("src/test/resources/conf/reporting-task-flow.xml"));
@@ -182,7 +185,8 @@ public class TestPopularVoteFlowElection {
 
     @Test
     public void testDifferentPopulatedFlowsElection() throws IOException {
-        final FingerprintFactory fingerprintFactory = new FingerprintFactory(createEncryptorFromProperties(getNiFiProperties()));
+        final ExtensionManager extensionManager = new StandardExtensionDiscoveringManager();
+        final FingerprintFactory fingerprintFactory = new FingerprintFactory(createEncryptorFromProperties(getNiFiProperties()), extensionManager);
         final PopularVoteFlowElection election = new PopularVoteFlowElection(1, TimeUnit.MINUTES, 4, fingerprintFactory);
         final byte[] nonEmptyCandidateA = Files.readAllBytes(Paths.get("src/test/resources/conf/controller-service-flow.xml"));
         final byte[] nonEmptyCandidateB = Files.readAllBytes(Paths.get("src/test/resources/conf/reporting-task-flow.xml"));

http://git-wip-us.apache.org/repos/asf/nifi/blob/fdd8cdbb/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/integration/Cluster.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/integration/Cluster.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/integration/Cluster.java
index 370d6dc..48c3639 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/integration/Cluster.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/integration/Cluster.java
@@ -17,13 +17,6 @@
 
 package org.apache.nifi.cluster.integration;
 
-import java.io.IOException;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.TimeUnit;
 import org.apache.curator.RetryPolicy;
 import org.apache.curator.framework.CuratorFramework;
 import org.apache.curator.framework.CuratorFrameworkFactory;
@@ -34,10 +27,20 @@ import org.apache.nifi.cluster.coordination.flow.PopularVoteFlowElection;
 import org.apache.nifi.cluster.coordination.node.ClusterRoles;
 import org.apache.nifi.encrypt.StringEncryptor;
 import org.apache.nifi.fingerprint.FingerprintFactory;
+import org.apache.nifi.nar.ExtensionDiscoveringManager;
+import org.apache.nifi.nar.StandardExtensionDiscoveringManager;
 import org.apache.nifi.util.NiFiProperties;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.io.IOException;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+
 public class Cluster {
     private static final Logger logger = LoggerFactory.getLogger(Cluster.class);
 
@@ -134,10 +137,11 @@ public class Cluster {
         final String provider = nifiProperties.getProperty(NiFiProperties.SENSITIVE_PROPS_PROVIDER);
         final String password = nifiProperties.getProperty(NiFiProperties.SENSITIVE_PROPS_KEY);
         final StringEncryptor encryptor = StringEncryptor.createEncryptor(algorithm, provider, password);
-        final FingerprintFactory fingerprintFactory = new FingerprintFactory(encryptor);
+        final ExtensionDiscoveringManager extensionManager = new StandardExtensionDiscoveringManager();
+        final FingerprintFactory fingerprintFactory = new FingerprintFactory(encryptor, extensionManager);
         final FlowElection flowElection = new PopularVoteFlowElection(flowElectionTimeoutMillis, TimeUnit.MILLISECONDS, flowElectionMaxNodes, fingerprintFactory);
 
-        final Node node = new Node(nifiProperties, flowElection);
+        final Node node = new Node(nifiProperties, extensionManager, flowElection);
         node.start();
         nodes.add(node);
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/fdd8cdbb/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 b2a499a..ea181ac 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
@@ -51,6 +51,7 @@ import org.apache.nifi.engine.FlowEngine;
 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.ExtensionDiscoveringManager;
 import org.apache.nifi.nar.ExtensionManager;
 import org.apache.nifi.nar.SystemBundle;
 import org.apache.nifi.registry.VariableRegistry;
@@ -77,6 +78,7 @@ import java.util.concurrent.TimeUnit;
 public class Node {
     private final NodeIdentifier nodeId;
     private final NiFiProperties nodeProperties;
+    private final ExtensionManager extensionManager;
 
     private final List<ReportedEvent> reportedEvents = Collections.synchronizedList(new ArrayList<ReportedEvent>());
     private final RevisionManager revisionManager;
@@ -95,11 +97,11 @@ public class Node {
     private ScheduledExecutorService executor = new FlowEngine(8, "Node tasks", true);
 
 
-    public Node(final NiFiProperties properties, final FlowElection flowElection) {
-        this(createNodeId(), properties, flowElection);
+    public Node(final NiFiProperties properties, final ExtensionDiscoveringManager extensionManager, final FlowElection flowElection) {
+        this(createNodeId(), properties, extensionManager, flowElection);
     }
 
-    public Node(final NodeIdentifier nodeId, final NiFiProperties properties, final FlowElection flowElection) {
+    public Node(final NodeIdentifier nodeId, final NiFiProperties properties, final ExtensionDiscoveringManager extensionManager, final FlowElection flowElection) {
         this.nodeId = nodeId;
         this.nodeProperties = new NiFiProperties() {
             @Override
@@ -125,7 +127,8 @@ public class Node {
         };
 
         final Bundle systemBundle = SystemBundle.create(properties);
-        ExtensionManager.discoverExtensions(systemBundle, Collections.emptySet());
+        extensionManager.discoverExtensions(systemBundle, Collections.emptySet());
+        this.extensionManager = extensionManager;
 
         revisionManager = Mockito.mock(RevisionManager.class);
         Mockito.when(revisionManager.getAllRevisions()).thenReturn(Collections.emptyList());
@@ -163,7 +166,7 @@ public class Node {
         final HeartbeatMonitor heartbeatMonitor = createHeartbeatMonitor();
         flowController = FlowController.createClusteredInstance(Mockito.mock(FlowFileEventRepository.class), nodeProperties,
             null, null, createEncryptorFromProperties(nodeProperties), protocolSender, Mockito.mock(BulletinRepository.class), clusterCoordinator,
-            heartbeatMonitor, electionManager, VariableRegistry.EMPTY_REGISTRY, Mockito.mock(FlowRegistryClient.class));
+            heartbeatMonitor, electionManager, VariableRegistry.EMPTY_REGISTRY, Mockito.mock(FlowRegistryClient.class), extensionManager);
 
         try {
             flowController.initializeFlow();
@@ -301,7 +304,7 @@ public class Node {
         final ClusterCoordinationProtocolSenderListener protocolSenderListener = new ClusterCoordinationProtocolSenderListener(createCoordinatorProtocolSender(), protocolListener);
         try {
             return new NodeClusterCoordinator(protocolSenderListener, eventReporter, electionManager, flowElection, null,
-                    revisionManager, nodeProperties, protocolSender);
+                    revisionManager, nodeProperties, extensionManager, protocolSender);
         } catch (IOException e) {
             Assert.fail(e.toString());
             return null;

http://git-wip-us.apache.org/repos/asf/nifi/blob/fdd8cdbb/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/AbstractComponentNode.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/AbstractComponentNode.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/AbstractComponentNode.java
index 42214a9..ab9ece0 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/AbstractComponentNode.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/AbstractComponentNode.java
@@ -71,6 +71,7 @@ public abstract class AbstractComponentNode implements ComponentNode {
     private final String componentCanonicalClass;
     private final ComponentVariableRegistry variableRegistry;
     private final ReloadComponent reloadComponent;
+    private final ExtensionManager extensionManager;
 
     private final AtomicBoolean isExtensionMissing;
 
@@ -82,9 +83,10 @@ public abstract class AbstractComponentNode implements ComponentNode {
     private volatile boolean triggerValidation = true;
 
     public AbstractComponentNode(final String id,
-                                       final ValidationContextFactory validationContextFactory, final ControllerServiceProvider serviceProvider,
-                                       final String componentType, final String componentCanonicalClass, final ComponentVariableRegistry variableRegistry,
-        final ReloadComponent reloadComponent, final ValidationTrigger validationTrigger, final boolean isExtensionMissing) {
+                                 final ValidationContextFactory validationContextFactory, final ControllerServiceProvider serviceProvider,
+                                 final String componentType, final String componentCanonicalClass, final ComponentVariableRegistry variableRegistry,
+                                 final ReloadComponent reloadComponent, final ExtensionManager extensionManager, final ValidationTrigger validationTrigger,
+                                 final boolean isExtensionMissing) {
         this.id = id;
         this.validationContextFactory = validationContextFactory;
         this.serviceProvider = serviceProvider;
@@ -94,6 +96,7 @@ public abstract class AbstractComponentNode implements ComponentNode {
         this.variableRegistry = variableRegistry;
         this.validationTrigger = validationTrigger;
         this.reloadComponent = reloadComponent;
+        this.extensionManager = extensionManager;
         this.isExtensionMissing = new AtomicBoolean(isExtensionMissing);
     }
 
@@ -171,7 +174,7 @@ public abstract class AbstractComponentNode implements ComponentNode {
         try {
             verifyModifiable();
 
-            try (final NarCloseable narCloseable = NarCloseable.withComponentNarLoader(getComponent().getClass(), id)) {
+            try (final NarCloseable narCloseable = NarCloseable.withComponentNarLoader(extensionManager, getComponent().getClass(), id)) {
                 boolean classpathChanged = false;
                 for (final Map.Entry<String, String> entry : properties.entrySet()) {
                     // determine if any of the property changes require resetting the InstanceClassLoader
@@ -285,7 +288,7 @@ public abstract class AbstractComponentNode implements ComponentNode {
 
     @Override
     public Map<PropertyDescriptor, String> getProperties() {
-        try (final NarCloseable narCloseable = NarCloseable.withComponentNarLoader(getComponent().getClass(), getIdentifier())) {
+        try (final NarCloseable narCloseable = NarCloseable.withComponentNarLoader(extensionManager, getComponent().getClass(), getIdentifier())) {
             final List<PropertyDescriptor> supported = getComponent().getPropertyDescriptors();
             if (supported == null || supported.isEmpty()) {
                 return Collections.unmodifiableMap(properties);
@@ -365,7 +368,7 @@ public abstract class AbstractComponentNode implements ComponentNode {
 
     @Override
     public String toString() {
-        try (final NarCloseable narCloseable = NarCloseable.withComponentNarLoader(getComponent().getClass(), getComponent().getIdentifier())) {
+        try (final NarCloseable narCloseable = NarCloseable.withComponentNarLoader(extensionManager, getComponent().getClass(), getComponent().getIdentifier())) {
             return getComponent().toString();
         }
     }
@@ -378,7 +381,7 @@ public abstract class AbstractComponentNode implements ComponentNode {
 
             final ValidationContext validationContext = getValidationContext();
             final Collection<ValidationResult> results = new ArrayList<>();
-            try (final NarCloseable narCloseable = NarCloseable.withComponentNarLoader(getComponent().getClass(), getIdentifier())) {
+            try (final NarCloseable narCloseable = NarCloseable.withComponentNarLoader(extensionManager, getComponent().getClass(), getIdentifier())) {
                 final Collection<ValidationResult> validationResults = computeValidationErrors(validationContext);
                 results.addAll(validationResults);
 
@@ -467,23 +470,24 @@ public abstract class AbstractComponentNode implements ComponentNode {
     private ValidationResult validateControllerServiceApi(final PropertyDescriptor descriptor, final ControllerServiceNode controllerServiceNode) {
         final Class<? extends ControllerService> controllerServiceApiClass = descriptor.getControllerServiceDefinition();
         final ClassLoader controllerServiceApiClassLoader = controllerServiceApiClass.getClassLoader();
+        final ExtensionManager extensionManager = serviceProvider.getExtensionManager();
 
         final String serviceId = controllerServiceNode.getIdentifier();
         final String propertyName = descriptor.getDisplayName();
 
-        final Bundle controllerServiceApiBundle = ExtensionManager.getBundle(controllerServiceApiClassLoader);
+        final Bundle controllerServiceApiBundle = extensionManager.getBundle(controllerServiceApiClassLoader);
         if (controllerServiceApiBundle == null) {
             return createInvalidResult(serviceId, propertyName, "Unable to find bundle for ControllerService API class " + controllerServiceApiClass.getCanonicalName());
         }
         final BundleCoordinate controllerServiceApiCoordinate = controllerServiceApiBundle.getBundleDetails().getCoordinate();
 
-        final Bundle controllerServiceBundle = ExtensionManager.getBundle(controllerServiceNode.getBundleCoordinate());
+        final Bundle controllerServiceBundle = extensionManager.getBundle(controllerServiceNode.getBundleCoordinate());
         if (controllerServiceBundle == null) {
             return createInvalidResult(serviceId, propertyName, "Unable to find bundle for coordinate " + controllerServiceNode.getBundleCoordinate());
         }
         final BundleCoordinate controllerServiceCoordinate = controllerServiceBundle.getBundleDetails().getCoordinate();
 
-        final boolean matchesApi = matchesApi(controllerServiceBundle, controllerServiceApiCoordinate);
+        final boolean matchesApi = matchesApi(extensionManager, controllerServiceBundle, controllerServiceApiCoordinate);
 
         if (!matchesApi) {
             final String controllerServiceType = controllerServiceNode.getComponentType();
@@ -518,7 +522,7 @@ public abstract class AbstractComponentNode implements ComponentNode {
      * @param requiredApiCoordinate the controller service API required by the processor
      * @return true if the controller service node has the require API as an ancestor, false otherwise
      */
-    private boolean matchesApi(final Bundle controllerServiceImplBundle, final BundleCoordinate requiredApiCoordinate) {
+    private boolean matchesApi(final ExtensionManager extensionManager, final Bundle controllerServiceImplBundle, final BundleCoordinate requiredApiCoordinate) {
         // start with the coordinate of the controller service for cases where the API and service are in the same bundle
         BundleCoordinate controllerServiceDependencyCoordinate = controllerServiceImplBundle.getBundleDetails().getCoordinate();
 
@@ -531,7 +535,7 @@ public abstract class AbstractComponentNode implements ComponentNode {
             }
 
             // move to the next dependency in the chain, or stop if null
-            final Bundle controllerServiceDependencyBundle = ExtensionManager.getBundle(controllerServiceDependencyCoordinate);
+            final Bundle controllerServiceDependencyBundle = extensionManager.getBundle(controllerServiceDependencyCoordinate);
             if (controllerServiceDependencyBundle == null) {
                 controllerServiceDependencyCoordinate = null;
             } else {
@@ -544,21 +548,21 @@ public abstract class AbstractComponentNode implements ComponentNode {
 
     @Override
     public PropertyDescriptor getPropertyDescriptor(final String name) {
-        try (final NarCloseable narCloseable = NarCloseable.withComponentNarLoader(getComponent().getClass(), getComponent().getIdentifier())) {
+        try (final NarCloseable narCloseable = NarCloseable.withComponentNarLoader(extensionManager, getComponent().getClass(), getComponent().getIdentifier())) {
             return getComponent().getPropertyDescriptor(name);
         }
     }
 
     @Override
     public List<PropertyDescriptor> getPropertyDescriptors() {
-        try (final NarCloseable narCloseable = NarCloseable.withComponentNarLoader(getComponent().getClass(), getComponent().getIdentifier())) {
+        try (final NarCloseable narCloseable = NarCloseable.withComponentNarLoader(extensionManager, getComponent().getClass(), getComponent().getIdentifier())) {
             return getComponent().getPropertyDescriptors();
         }
     }
 
 
     private final void onPropertyModified(final PropertyDescriptor descriptor, final String oldValue, final String newValue) {
-        try (final NarCloseable narCloseable = NarCloseable.withComponentNarLoader(getComponent().getClass(), getComponent().getIdentifier())) {
+        try (final NarCloseable narCloseable = NarCloseable.withComponentNarLoader(extensionManager, getComponent().getClass(), getComponent().getIdentifier())) {
             getComponent().onPropertyModified(descriptor, oldValue, newValue);
         }
     }
@@ -734,6 +738,10 @@ public abstract class AbstractComponentNode implements ComponentNode {
         return this.reloadComponent;
     }
 
+    protected ExtensionManager getExtensionManager() {
+        return this.extensionManager;
+    }
+
     @Override
     public void verifyCanUpdateBundle(final BundleCoordinate incomingCoordinate) throws IllegalArgumentException {
         final BundleCoordinate existingCoordinate = getBundleCoordinate();

http://git-wip-us.apache.org/repos/asf/nifi/blob/fdd8cdbb/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 0da86d3..f3adab0 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
@@ -32,6 +32,7 @@ import org.apache.nifi.controller.scheduling.SchedulingAgent;
 import org.apache.nifi.controller.service.ControllerServiceNode;
 import org.apache.nifi.controller.service.ControllerServiceProvider;
 import org.apache.nifi.logging.LogLevel;
+import org.apache.nifi.nar.ExtensionManager;
 import org.apache.nifi.processor.ProcessContext;
 import org.apache.nifi.processor.Processor;
 import org.apache.nifi.processor.Relationship;
@@ -46,8 +47,10 @@ public abstract class ProcessorNode extends AbstractComponentNode implements Con
     public ProcessorNode(final String id,
                          final ValidationContextFactory validationContextFactory, final ControllerServiceProvider serviceProvider,
                          final String componentType, final String componentCanonicalClass, final ComponentVariableRegistry variableRegistry,
-                         final ReloadComponent reloadComponent, final ValidationTrigger validationTrigger, final boolean isExtensionMissing) {
-        super(id, validationContextFactory, serviceProvider, componentType, componentCanonicalClass, variableRegistry, reloadComponent, validationTrigger, isExtensionMissing);
+                         final ReloadComponent reloadComponent, final ExtensionManager extensionManager, final ValidationTrigger validationTrigger,
+                         final boolean isExtensionMissing) {
+        super(id, validationContextFactory, serviceProvider, componentType, componentCanonicalClass, variableRegistry, reloadComponent,
+                extensionManager, validationTrigger, isExtensionMissing);
         this.scheduledState = new AtomicReference<>(ScheduledState.STOPPED);
     }
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/fdd8cdbb/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 c826775..4e310b7 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
@@ -20,6 +20,7 @@ import java.util.Set;
 
 import org.apache.nifi.bundle.BundleCoordinate;
 import org.apache.nifi.controller.ReportingTaskNode;
+import org.apache.nifi.nar.ExtensionManager;
 
 /**
  * A ReportingTaskProvider is responsible for providing management of, and
@@ -112,4 +113,9 @@ public interface ReportingTaskProvider {
      */
     void disableReportingTask(ReportingTaskNode reportingTask);
 
+    /**
+     * @return the ExtensionManager instance used by this provider
+     */
+    ExtensionManager getExtensionManager();
+
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/fdd8cdbb/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 56276f4..95eb6a5 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
@@ -27,6 +27,7 @@ import org.apache.nifi.bundle.BundleCoordinate;
 import org.apache.nifi.controller.ComponentNode;
 import org.apache.nifi.controller.ControllerService;
 import org.apache.nifi.controller.ControllerServiceLookup;
+import org.apache.nifi.nar.ExtensionManager;
 
 /**
  *
@@ -225,4 +226,10 @@ public interface ControllerServiceProvider extends ControllerServiceLookup {
      *         identifier
      */
     ControllerService getControllerServiceForComponent(String serviceIdentifier, String componentId);
+
+    /**
+     * @return the ExtensionManager used by this provider
+     */
+    ExtensionManager getExtensionManager();
+
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/fdd8cdbb/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/test/java/org/apache/nifi/controller/TestAbstractComponentNode.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/test/java/org/apache/nifi/controller/TestAbstractComponentNode.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/test/java/org/apache/nifi/controller/TestAbstractComponentNode.java
index 3fa5d2c..49dfb8b 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/test/java/org/apache/nifi/controller/TestAbstractComponentNode.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/test/java/org/apache/nifi/controller/TestAbstractComponentNode.java
@@ -35,6 +35,7 @@ import org.apache.nifi.components.ValidationResult;
 import org.apache.nifi.components.validation.ValidationStatus;
 import org.apache.nifi.components.validation.ValidationTrigger;
 import org.apache.nifi.controller.service.ControllerServiceProvider;
+import org.apache.nifi.nar.StandardExtensionDiscoveringManager;
 import org.apache.nifi.registry.ComponentVariableRegistry;
 import org.junit.Test;
 import org.mockito.Mockito;
@@ -85,7 +86,7 @@ public class TestAbstractComponentNode {
         public ValidationControlledAbstractComponentNode(final long pauseMillis, final ValidationTrigger validationTrigger) {
             super("id", Mockito.mock(ValidationContextFactory.class), Mockito.mock(ControllerServiceProvider.class), "unit test component",
                 ValidationControlledAbstractComponentNode.class.getCanonicalName(), Mockito.mock(ComponentVariableRegistry.class), Mockito.mock(ReloadComponent.class),
-                validationTrigger, false);
+                Mockito.mock(StandardExtensionDiscoveringManager.class), validationTrigger, false);
 
             this.pauseMillis = pauseMillis;
         }


[6/7] nifi git commit: NIFI-5673 Set up property/assembly for new auto-load directory - Set up NarAutoLoader to watch directory for new files - Move NarAutoLoader to JettyServer since it will need access to ExtensionManager - Created NarLoader to shared

Posted by ma...@apache.org.
http://git-wip-us.apache.org/repos/asf/nifi/blob/fdd8cdbb/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 69bdf2a..680962e 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
@@ -441,7 +441,8 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
             final StringEncryptor encryptor,
             final BulletinRepository bulletinRepo,
             final VariableRegistry variableRegistry,
-            final FlowRegistryClient flowRegistryClient) {
+            final FlowRegistryClient flowRegistryClient,
+            final ExtensionManager extensionManager) {
 
         return new FlowController(
                 flowFileEventRepo,
@@ -456,7 +457,8 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
                 /* heartbeat monitor */ null,
                 /* leader election manager */ null,
                 /* variable registry */ variableRegistry,
-                flowRegistryClient);
+                flowRegistryClient,
+                extensionManager);
     }
 
     public static FlowController createClusteredInstance(
@@ -471,7 +473,8 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
             final HeartbeatMonitor heartbeatMonitor,
             final LeaderElectionManager leaderElectionManager,
             final VariableRegistry variableRegistry,
-            final FlowRegistryClient flowRegistryClient) {
+            final FlowRegistryClient flowRegistryClient,
+            final ExtensionManager extensionManager) {
 
         final FlowController flowController = new FlowController(
                 flowFileEventRepo,
@@ -486,7 +489,8 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
                 heartbeatMonitor,
                 leaderElectionManager,
                 variableRegistry,
-                flowRegistryClient);
+                flowRegistryClient,
+                extensionManager);
 
         return flowController;
     }
@@ -505,7 +509,8 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
             final HeartbeatMonitor heartbeatMonitor,
             final LeaderElectionManager leaderElectionManager,
             final VariableRegistry variableRegistry,
-            final FlowRegistryClient flowRegistryClient) {
+            final FlowRegistryClient flowRegistryClient,
+            final ExtensionManager extensionManager) {
 
         maxTimerDrivenThreads = new AtomicInteger(10);
         maxEventDrivenThreads = new AtomicInteger(5);
@@ -513,14 +518,14 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
         this.encryptor = encryptor;
         this.nifiProperties = nifiProperties;
         this.heartbeatMonitor = heartbeatMonitor;
-        sslContext = SslContextFactory.createSslContext(nifiProperties);
-        extensionManager = new ExtensionManager();
+        this.sslContext = SslContextFactory.createSslContext(nifiProperties);
+        this.extensionManager = extensionManager;
         this.clusterCoordinator = clusterCoordinator;
 
         timerDrivenEngineRef = new AtomicReference<>(new FlowEngine(maxTimerDrivenThreads.get(), "Timer-Driven Process"));
         eventDrivenEngineRef = new AtomicReference<>(new FlowEngine(maxEventDrivenThreads.get(), "Event-Driven Process"));
 
-        final FlowFileRepository flowFileRepo = createFlowFileRepository(nifiProperties, resourceClaimManager);
+        final FlowFileRepository flowFileRepo = createFlowFileRepository(nifiProperties, extensionManager, resourceClaimManager);
         flowFileRepository = flowFileRepo;
         flowFileEventRepository = flowFileEventRepo;
         counterRepositoryRef = new AtomicReference<>(new StandardCounterRepository());
@@ -542,7 +547,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
         }
 
         try {
-            this.stateManagerProvider = StandardStateManagerProvider.create(nifiProperties, this.variableRegistry);
+            this.stateManagerProvider = StandardStateManagerProvider.create(nifiProperties, this.variableRegistry, extensionManager);
         } catch (final IOException e) {
             throw new RuntimeException(e);
         }
@@ -553,7 +558,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
         final RepositoryContextFactory contextFactory = new RepositoryContextFactory(contentRepository, flowFileRepository, flowFileEventRepository, counterRepositoryRef.get(), provenanceRepository);
 
         eventDrivenSchedulingAgent = new EventDrivenSchedulingAgent(
-            eventDrivenEngineRef.get(), this, stateManagerProvider, eventDrivenWorkerQueue, contextFactory, maxEventDrivenThreads.get(), encryptor);
+            eventDrivenEngineRef.get(), this, stateManagerProvider, eventDrivenWorkerQueue, contextFactory, maxEventDrivenThreads.get(), encryptor, extensionManager);
         processScheduler.setSchedulingAgent(SchedulingStrategy.EVENT_DRIVEN, eventDrivenSchedulingAgent);
 
         final QuartzSchedulingAgent quartzSchedulingAgent = new QuartzSchedulingAgent(this, timerDrivenEngineRef.get(), contextFactory, encryptor);
@@ -743,7 +748,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
         return ResourceFactory.getControllerResource();
     }
 
-    private static FlowFileRepository createFlowFileRepository(final NiFiProperties properties, final ResourceClaimManager contentClaimManager) {
+    private static FlowFileRepository createFlowFileRepository(final NiFiProperties properties, final ExtensionManager extensionManager, final ResourceClaimManager contentClaimManager) {
         final String implementationClassName = properties.getProperty(NiFiProperties.FLOWFILE_REPOSITORY_IMPLEMENTATION, DEFAULT_FLOWFILE_REPO_IMPLEMENTATION);
         if (implementationClassName == null) {
             throw new RuntimeException("Cannot create FlowFile Repository because the NiFi Properties is missing the following property: "
@@ -751,7 +756,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
         }
 
         try {
-            final FlowFileRepository created = NarThreadContextClassLoader.createInstance(implementationClassName, FlowFileRepository.class, properties);
+            final FlowFileRepository created = NarThreadContextClassLoader.createInstance(extensionManager, implementationClassName, FlowFileRepository.class, properties);
             synchronized (created) {
                 created.initialize(contentClaimManager);
             }
@@ -761,14 +766,14 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
         }
     }
 
-    private static FlowFileSwapManager createSwapManager(final NiFiProperties properties) {
+    private static FlowFileSwapManager createSwapManager(final NiFiProperties properties, final ExtensionManager extensionManager) {
         final String implementationClassName = properties.getProperty(NiFiProperties.FLOWFILE_SWAP_MANAGER_IMPLEMENTATION, DEFAULT_SWAP_MANAGER_IMPLEMENTATION);
         if (implementationClassName == null) {
             return null;
         }
 
         try {
-            return NarThreadContextClassLoader.createInstance(implementationClassName, FlowFileSwapManager.class, properties);
+            return NarThreadContextClassLoader.createInstance(extensionManager, implementationClassName, FlowFileSwapManager.class, properties);
         } catch (final Exception e) {
             throw new RuntimeException(e);
         }
@@ -876,7 +881,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
     private void notifyComponentsConfigurationRestored() {
         for (final ProcessorNode procNode : getGroup(getRootGroupId()).findAllProcessors()) {
             final Processor processor = procNode.getProcessor();
-            try (final NarCloseable nc = NarCloseable.withComponentNarLoader(processor.getClass(), processor.getIdentifier())) {
+            try (final NarCloseable nc = NarCloseable.withComponentNarLoader(extensionManager, processor.getClass(), processor.getIdentifier())) {
                 ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnConfigurationRestored.class, processor);
             }
         }
@@ -884,7 +889,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
         for (final ControllerServiceNode serviceNode : getAllControllerServices()) {
             final ControllerService service = serviceNode.getControllerServiceImplementation();
 
-            try (final NarCloseable nc = NarCloseable.withComponentNarLoader(service.getClass(), service.getIdentifier())) {
+            try (final NarCloseable nc = NarCloseable.withComponentNarLoader(extensionManager, service.getClass(), service.getIdentifier())) {
                 ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnConfigurationRestored.class, service);
             }
         }
@@ -892,7 +897,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
         for (final ReportingTaskNode taskNode : getAllReportingTasks()) {
             final ReportingTask task = taskNode.getReportingTask();
 
-            try (final NarCloseable nc = NarCloseable.withComponentNarLoader(task.getClass(), task.getIdentifier())) {
+            try (final NarCloseable nc = NarCloseable.withComponentNarLoader(extensionManager, task.getClass(), task.getIdentifier())) {
                 ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnConfigurationRestored.class, task);
             }
         }
@@ -1020,7 +1025,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
         }
 
         try {
-            final ContentRepository contentRepo = NarThreadContextClassLoader.createInstance(implementationClassName, ContentRepository.class, properties);
+            final ContentRepository contentRepo = NarThreadContextClassLoader.createInstance(extensionManager, implementationClassName, ContentRepository.class, properties);
             synchronized (contentRepo) {
                 contentRepo.initialize(resourceClaimManager);
             }
@@ -1038,7 +1043,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
         }
 
         try {
-            return NarThreadContextClassLoader.createInstance(implementationClassName, ProvenanceRepository.class, properties);
+            return NarThreadContextClassLoader.createInstance(extensionManager, implementationClassName, ProvenanceRepository.class, properties);
         } catch (final Exception e) {
             throw new RuntimeException(e);
         }
@@ -1052,7 +1057,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
         }
 
         try {
-            return NarThreadContextClassLoader.createInstance(implementationClassName, ComponentStatusRepository.class, nifiProperties);
+            return NarThreadContextClassLoader.createInstance(extensionManager, implementationClassName, ComponentStatusRepository.class, nifiProperties);
         } catch (final Exception e) {
             throw new RuntimeException(e);
         }
@@ -1083,7 +1088,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
         }
 
         // Create and initialize a FlowFileSwapManager for this connection
-        final FlowFileSwapManager swapManager = createSwapManager(nifiProperties);
+        final FlowFileSwapManager swapManager = createSwapManager(nifiProperties, extensionManager);
         final EventReporter eventReporter = createEventReporter(getBulletinRepository());
 
         try (final NarCloseable narCloseable = NarCloseable.withNarLoader()) {
@@ -1288,12 +1293,12 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
         final ProcessorNode procNode;
         if (creationSuccessful) {
             procNode = new StandardProcessorNode(processor, id, validationContextFactory, processScheduler, controllerServiceProvider,
-                nifiProperties, componentVarRegistry, this, validationTrigger);
+                nifiProperties, componentVarRegistry, this, extensionManager, validationTrigger);
         } 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, componentVarRegistry, this, validationTrigger, true);
+                componentType, type, nifiProperties, componentVarRegistry, this, extensionManager, validationTrigger, true);
         }
 
         if (registerLogObserver) {
@@ -1331,7 +1336,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
         }
 
         if (firstTimeAdded) {
-            try (final NarCloseable x = NarCloseable.withComponentNarLoader(procNode.getProcessor().getClass(), procNode.getProcessor().getIdentifier())) {
+            try (final NarCloseable x = NarCloseable.withComponentNarLoader(extensionManager, procNode.getProcessor().getClass(), procNode.getProcessor().getIdentifier())) {
                 ReflectionUtils.invokeMethodsWithAnnotation(OnAdded.class, procNode.getProcessor());
             } catch (final Exception e) {
                 if (registerLogObserver) {
@@ -1341,7 +1346,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
             }
 
             if (firstTimeAdded) {
-                try (final NarCloseable nc = NarCloseable.withComponentNarLoader(procNode.getProcessor().getClass(), procNode.getProcessor().getIdentifier())) {
+                try (final NarCloseable nc = NarCloseable.withComponentNarLoader(extensionManager, procNode.getProcessor().getClass(), procNode.getProcessor().getIdentifier())) {
                     ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnConfigurationRestored.class, procNode.getProcessor());
                 }
             }
@@ -1353,14 +1358,14 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
     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);
+        final Bundle processorBundle = extensionManager.getBundle(bundleCoordinate);
         if (processorBundle == null) {
             throw new ProcessorInstantiationException("Unable to find bundle for coordinate " + bundleCoordinate.getCoordinate());
         }
 
         final ClassLoader ctxClassLoader = Thread.currentThread().getContextClassLoader();
         try {
-            final ClassLoader detectedClassLoaderForInstance = ExtensionManager.createInstanceClassLoader(type, identifier, processorBundle, additionalUrls);
+            final ClassLoader detectedClassLoaderForInstance = extensionManager.createInstanceClassLoader(type, identifier, processorBundle, additionalUrls);
             final Class<?> rawClass = Class.forName(type, true, detectedClassLoaderForInstance);
             Thread.currentThread().setContextClassLoader(detectedClassLoaderForInstance);
 
@@ -1400,7 +1405,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
 
         // createProcessor will create a new instance class loader for the same id so
         // save the instance class loader to use it for calling OnRemoved on the existing processor
-        final ClassLoader existingInstanceClassLoader = ExtensionManager.getInstanceClassLoader(id);
+        final ClassLoader existingInstanceClassLoader = extensionManager.getInstanceClassLoader(id);
 
         // 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
@@ -1412,7 +1417,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
             final StandardProcessContext processContext = new StandardProcessContext(existingNode, controllerServiceProvider, encryptor, stateManager, () -> false);
             ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnRemoved.class, existingNode.getProcessor(), processContext);
         } finally {
-            ExtensionManager.closeURLClassLoader(id, existingInstanceClassLoader);
+            extensionManager.closeURLClassLoader(id, existingInstanceClassLoader);
         }
 
         // set the new processor in the existing node
@@ -1655,7 +1660,8 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
 
             // invoke any methods annotated with @OnShutdown on Controller Services
             for (final ControllerServiceNode serviceNode : getAllControllerServices()) {
-                try (final NarCloseable narCloseable = NarCloseable.withComponentNarLoader(serviceNode.getControllerServiceImplementation().getClass(), serviceNode.getIdentifier())) {
+                try (final NarCloseable narCloseable = NarCloseable.withComponentNarLoader(
+                        extensionManager, serviceNode.getControllerServiceImplementation().getClass(), serviceNode.getIdentifier())) {
                     final ConfigurationContext configContext = new StandardConfigurationContext(serviceNode, controllerServiceProvider, null, variableRegistry);
                     ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnShutdown.class, serviceNode.getControllerServiceImplementation(), configContext);
                 }
@@ -1664,7 +1670,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
             // invoke any methods annotated with @OnShutdown on Reporting Tasks
             for (final ReportingTaskNode taskNode : getAllReportingTasks()) {
                 final ConfigurationContext configContext = taskNode.getConfigurationContext();
-                try (final NarCloseable narCloseable = NarCloseable.withComponentNarLoader(taskNode.getReportingTask().getClass(), taskNode.getIdentifier())) {
+                try (final NarCloseable narCloseable = NarCloseable.withComponentNarLoader(extensionManager, taskNode.getReportingTask().getClass(), taskNode.getIdentifier())) {
                     ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnShutdown.class, taskNode.getReportingTask(), configContext);
                 }
             }
@@ -1987,7 +1993,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
             final List<ControllerServiceNode> serviceNodes = new ArrayList<>();
             try {
                 for (final ControllerServiceDTO controllerServiceDTO : dto.getControllerServices()) {
-                    final BundleCoordinate bundleCoordinate = BundleUtils.getBundle(controllerServiceDTO.getType(), controllerServiceDTO.getBundle());
+                    final BundleCoordinate bundleCoordinate = BundleUtils.getBundle(extensionManager, controllerServiceDTO.getType(), controllerServiceDTO.getBundle());
                     final ControllerServiceNode serviceNode = createControllerService(controllerServiceDTO.getType(), controllerServiceDTO.getId(), bundleCoordinate, Collections.emptySet(), true);
                     serviceNode.pauseValidationTrigger();
                     serviceNodes.add(serviceNode);
@@ -2097,7 +2103,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
             // Instantiate the processors
             //
             for (final ProcessorDTO processorDTO : dto.getProcessors()) {
-                final BundleCoordinate bundleCoordinate = BundleUtils.getBundle(processorDTO.getType(), processorDTO.getBundle());
+                final BundleCoordinate bundleCoordinate = BundleUtils.getBundle(extensionManager, processorDTO.getType(), processorDTO.getBundle());
                 final ProcessorNode procNode = createProcessor(processorDTO.getType(), processorDTO.getId(), bundleCoordinate);
                 procNode.pauseValidationTrigger();
 
@@ -2489,21 +2495,21 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
 
     public void verifyComponentTypesInSnippet(final FlowSnippetDTO templateContents) {
         final Map<String, Set<BundleCoordinate>> processorClasses = new HashMap<>();
-        for (final Class<?> c : ExtensionManager.getExtensions(Processor.class)) {
+        for (final Class<?> c : extensionManager.getExtensions(Processor.class)) {
             final String name = c.getName();
-            processorClasses.put(name, ExtensionManager.getBundles(name).stream().map(bundle -> bundle.getBundleDetails().getCoordinate()).collect(Collectors.toSet()));
+            processorClasses.put(name, extensionManager.getBundles(name).stream().map(bundle -> bundle.getBundleDetails().getCoordinate()).collect(Collectors.toSet()));
         }
         verifyProcessorsInSnippet(templateContents, processorClasses);
 
         final Map<String, Set<BundleCoordinate>> controllerServiceClasses = new HashMap<>();
-        for (final Class<?> c : ExtensionManager.getExtensions(ControllerService.class)) {
+        for (final Class<?> c : extensionManager.getExtensions(ControllerService.class)) {
             final String name = c.getName();
-            controllerServiceClasses.put(name, ExtensionManager.getBundles(name).stream().map(bundle -> bundle.getBundleDetails().getCoordinate()).collect(Collectors.toSet()));
+            controllerServiceClasses.put(name, extensionManager.getBundles(name).stream().map(bundle -> bundle.getBundleDetails().getCoordinate()).collect(Collectors.toSet()));
         }
         verifyControllerServicesInSnippet(templateContents, controllerServiceClasses);
 
         final Set<String> prioritizerClasses = new HashSet<>();
-        for (final Class<?> c : ExtensionManager.getExtensions(FlowFilePrioritizer.class)) {
+        for (final Class<?> c : extensionManager.getExtensions(FlowFilePrioritizer.class)) {
             prioritizerClasses.add(c.getName());
         }
 
@@ -2527,21 +2533,21 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
 
     public void verifyComponentTypesInSnippet(final VersionedProcessGroup versionedFlow) {
         final Map<String, Set<BundleCoordinate>> processorClasses = new HashMap<>();
-        for (final Class<?> c : ExtensionManager.getExtensions(Processor.class)) {
+        for (final Class<?> c : extensionManager.getExtensions(Processor.class)) {
             final String name = c.getName();
-            processorClasses.put(name, ExtensionManager.getBundles(name).stream().map(bundle -> bundle.getBundleDetails().getCoordinate()).collect(Collectors.toSet()));
+            processorClasses.put(name, extensionManager.getBundles(name).stream().map(bundle -> bundle.getBundleDetails().getCoordinate()).collect(Collectors.toSet()));
         }
         verifyProcessorsInVersionedFlow(versionedFlow, processorClasses);
 
         final Map<String, Set<BundleCoordinate>> controllerServiceClasses = new HashMap<>();
-        for (final Class<?> c : ExtensionManager.getExtensions(ControllerService.class)) {
+        for (final Class<?> c : extensionManager.getExtensions(ControllerService.class)) {
             final String name = c.getName();
-            controllerServiceClasses.put(name, ExtensionManager.getBundles(name).stream().map(bundle -> bundle.getBundleDetails().getCoordinate()).collect(Collectors.toSet()));
+            controllerServiceClasses.put(name, extensionManager.getBundles(name).stream().map(bundle -> bundle.getBundleDetails().getCoordinate()).collect(Collectors.toSet()));
         }
         verifyControllerServicesInVersionedFlow(versionedFlow, controllerServiceClasses);
 
         final Set<String> prioritizerClasses = new HashSet<>();
-        for (final Class<?> c : ExtensionManager.getExtensions(FlowFilePrioritizer.class)) {
+        for (final Class<?> c : extensionManager.getExtensions(FlowFilePrioritizer.class)) {
             prioritizerClasses.add(c.getName());
         }
 
@@ -2666,7 +2672,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
 
         final ClassLoader ctxClassLoader = Thread.currentThread().getContextClassLoader();
         try {
-            final List<Bundle> prioritizerBundles = ExtensionManager.getBundles(type);
+            final List<Bundle> prioritizerBundles = extensionManager.getBundles(type);
             if (prioritizerBundles.size() == 0) {
                 throw new IllegalStateException(String.format("The specified class '%s' is not known to this nifi.", type));
             }
@@ -2754,12 +2760,12 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
     //
     @SuppressWarnings("rawtypes")
     public Set<Class> getFlowFileProcessorClasses() {
-        return ExtensionManager.getExtensions(Processor.class);
+        return extensionManager.getExtensions(Processor.class);
     }
 
     @SuppressWarnings("rawtypes")
     public Set<Class> getFlowFileComparatorClasses() {
-        return ExtensionManager.getExtensions(FlowFilePrioritizer.class);
+        return extensionManager.getExtensions(FlowFilePrioritizer.class);
     }
 
     /**
@@ -3677,12 +3683,14 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
         final ValidationContextFactory validationContextFactory = new StandardValidationContextFactory(controllerServiceProvider, componentVarRegistry);
         final ReportingTaskNode taskNode;
         if (creationSuccessful) {
-            taskNode = new StandardReportingTaskNode(task, id, this, processScheduler, validationContextFactory, componentVarRegistry, this, validationTrigger);
+            taskNode = new StandardReportingTaskNode(task, id, this, processScheduler, validationContextFactory, componentVarRegistry,
+                    this, extensionManager, validationTrigger);
         } 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, componentVarRegistry, this, validationTrigger, true);
+            taskNode = new StandardReportingTaskNode(task, id, this, processScheduler, validationContextFactory, componentType, type, componentVarRegistry,
+                    this, extensionManager, validationTrigger, true);
         }
 
         taskNode.setName(taskNode.getReportingTask().getClass().getSimpleName());
@@ -3697,7 +3705,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
                 throw new ReportingTaskInstantiationException("Failed to initialize reporting task of type " + type, ie);
             }
 
-            try (final NarCloseable x = NarCloseable.withComponentNarLoader(taskNode.getReportingTask().getClass(), taskNode.getReportingTask().getIdentifier())) {
+            try (final NarCloseable x = NarCloseable.withComponentNarLoader(extensionManager, taskNode.getReportingTask().getClass(), taskNode.getReportingTask().getIdentifier())) {
                 ReflectionUtils.invokeMethodsWithAnnotation(OnAdded.class, taskNode.getReportingTask());
                 ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnConfigurationRestored.class, taskNode.getReportingTask());
             } catch (final Exception e) {
@@ -3721,12 +3729,12 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
 
         final ClassLoader ctxClassLoader = Thread.currentThread().getContextClassLoader();
         try {
-            final Bundle reportingTaskBundle = ExtensionManager.getBundle(bundleCoordinate);
+            final Bundle reportingTaskBundle = extensionManager.getBundle(bundleCoordinate);
             if (reportingTaskBundle == null) {
                 throw new IllegalStateException("Unable to find bundle for coordinate " + bundleCoordinate.getCoordinate());
             }
 
-            final ClassLoader detectedClassLoader = ExtensionManager.createInstanceClassLoader(type, id, reportingTaskBundle, additionalUrls);
+            final ClassLoader detectedClassLoader = extensionManager.createInstanceClassLoader(type, id, reportingTaskBundle, additionalUrls);
             final Class<?> rawClass = Class.forName(type, false, detectedClassLoader);
             Thread.currentThread().setContextClassLoader(detectedClassLoader);
 
@@ -3763,7 +3771,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
 
         // createReportingTask will create a new instance class loader for the same id so
         // save the instance class loader to use it for calling OnRemoved on the existing processor
-        final ClassLoader existingInstanceClassLoader = ExtensionManager.getInstanceClassLoader(id);
+        final ClassLoader existingInstanceClassLoader = extensionManager.getInstanceClassLoader(id);
 
         // 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
@@ -3773,7 +3781,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
         try (final NarCloseable x = NarCloseable.withComponentNarLoader(existingInstanceClassLoader)) {
             ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnRemoved.class, existingNode.getReportingTask(), existingNode.getConfigurationContext());
         } finally {
-            ExtensionManager.closeURLClassLoader(id, existingInstanceClassLoader);
+            extensionManager.closeURLClassLoader(id, existingInstanceClassLoader);
         }
 
         // set the new reporting task into the existing node
@@ -3828,7 +3836,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
 
         reportingTaskNode.verifyCanDelete();
 
-        try (final NarCloseable x = NarCloseable.withComponentNarLoader(reportingTaskNode.getReportingTask().getClass(), reportingTaskNode.getReportingTask().getIdentifier())) {
+        try (final NarCloseable x = NarCloseable.withComponentNarLoader(extensionManager, reportingTaskNode.getReportingTask().getClass(), reportingTaskNode.getReportingTask().getIdentifier())) {
             ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnRemoved.class, reportingTaskNode.getReportingTask(), reportingTaskNode.getConfigurationContext());
         }
 
@@ -3849,7 +3857,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
         LogRepositoryFactory.removeRepository(reportingTaskNode.getIdentifier());
         processScheduler.onReportingTaskRemoved(reportingTaskNode);
 
-        ExtensionManager.removeInstanceClassLoader(reportingTaskNode.getIdentifier());
+        extensionManager.removeInstanceClassLoader(reportingTaskNode.getIdentifier());
     }
 
     @Override
@@ -3875,7 +3883,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
         if (firstTimeAdded) {
             final ControllerService service = serviceNode.getControllerServiceImplementation();
 
-            try (final NarCloseable nc = NarCloseable.withComponentNarLoader(service.getClass(), service.getIdentifier())) {
+            try (final NarCloseable nc = NarCloseable.withComponentNarLoader(extensionManager, service.getClass(), service.getIdentifier())) {
                 ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnConfigurationRestored.class, service);
             }
         }
@@ -3899,7 +3907,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
 
         // createControllerService will create a new instance class loader for the same id so
         // save the instance class loader to use it for calling OnRemoved on the existing service
-        final ClassLoader existingInstanceClassLoader = ExtensionManager.getInstanceClassLoader(id);
+        final ClassLoader existingInstanceClassLoader = extensionManager.getInstanceClassLoader(id);
 
         // 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
@@ -3910,7 +3918,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
             final ConfigurationContext configurationContext = new StandardConfigurationContext(existingNode, controllerServiceProvider, null, variableRegistry);
             ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnRemoved.class, existingNode.getControllerServiceImplementation(), configurationContext);
         } finally {
-            ExtensionManager.closeURLClassLoader(id, existingInstanceClassLoader);
+            extensionManager.closeURLClassLoader(id, existingInstanceClassLoader);
         }
 
         // take the invocation handler that was created for new proxy and is set to look at the new node,
@@ -4059,7 +4067,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
 
         service.verifyCanDelete();
 
-        try (final NarCloseable x = NarCloseable.withComponentNarLoader(service.getControllerServiceImplementation().getClass(), service.getIdentifier())) {
+        try (final NarCloseable x = NarCloseable.withComponentNarLoader(extensionManager, service.getControllerServiceImplementation().getClass(), service.getIdentifier())) {
             final ConfigurationContext configurationContext = new StandardConfigurationContext(service, controllerServiceProvider, null, variableRegistry);
             ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnRemoved.class, service.getControllerServiceImplementation(), configurationContext);
         }
@@ -4080,7 +4088,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
         rootControllerServices.remove(service.getIdentifier());
         getStateManagerProvider().onComponentRemoved(service.getIdentifier());
 
-        ExtensionManager.removeInstanceClassLoader(service.getIdentifier());
+        extensionManager.removeInstanceClassLoader(service.getIdentifier());
 
         LOG.info("{} removed from Flow Controller", service, this);
     }
@@ -4508,17 +4516,17 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
         final PrimaryNodeState nodeState = primary ? PrimaryNodeState.ELECTED_PRIMARY_NODE : PrimaryNodeState.PRIMARY_NODE_REVOKED;
         final ProcessGroup rootGroup = getGroup(getRootGroupId());
         for (final ProcessorNode procNode : rootGroup.findAllProcessors()) {
-            try (final NarCloseable narCloseable = NarCloseable.withComponentNarLoader(procNode.getProcessor().getClass(), procNode.getIdentifier())) {
+            try (final NarCloseable narCloseable = NarCloseable.withComponentNarLoader(extensionManager, procNode.getProcessor().getClass(), procNode.getIdentifier())) {
                 ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnPrimaryNodeStateChange.class, procNode.getProcessor(), nodeState);
             }
         }
         for (final ControllerServiceNode serviceNode : getAllControllerServices()) {
-            try (final NarCloseable narCloseable = NarCloseable.withComponentNarLoader(serviceNode.getControllerServiceImplementation().getClass(), serviceNode.getIdentifier())) {
+            try (final NarCloseable narCloseable = NarCloseable.withComponentNarLoader(extensionManager, serviceNode.getControllerServiceImplementation().getClass(), serviceNode.getIdentifier())) {
                 ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnPrimaryNodeStateChange.class, serviceNode.getControllerServiceImplementation(), nodeState);
             }
         }
         for (final ReportingTaskNode reportingTaskNode : getAllReportingTasks()) {
-            try (final NarCloseable narCloseable = NarCloseable.withComponentNarLoader(reportingTaskNode.getReportingTask().getClass(), reportingTaskNode.getIdentifier())) {
+            try (final NarCloseable narCloseable = NarCloseable.withComponentNarLoader(extensionManager, reportingTaskNode.getReportingTask().getClass(), reportingTaskNode.getIdentifier())) {
                 ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnPrimaryNodeStateChange.class, reportingTaskNode.getReportingTask(), nodeState);
             }
         }
@@ -4920,7 +4928,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
     @Override
     @SuppressWarnings("rawtypes")
     public List<String> getComponentTypes() {
-        final Set<Class> procClasses = ExtensionManager.getExtensions(Processor.class);
+        final Set<Class> procClasses = extensionManager.getExtensions(Processor.class);
         final List<String> componentTypes = new ArrayList<>(procClasses.size() + 2);
         componentTypes.add(ProvenanceEventRecord.REMOTE_INPUT_PORT_TYPE);
         componentTypes.add(ProvenanceEventRecord.REMOTE_OUTPUT_PORT_TYPE);

http://git-wip-us.apache.org/repos/asf/nifi/blob/fdd8cdbb/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardFlowService.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardFlowService.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardFlowService.java
index 957357b..a34d00b 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardFlowService.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardFlowService.java
@@ -56,7 +56,7 @@ import org.apache.nifi.groups.ProcessGroup;
 import org.apache.nifi.groups.RemoteProcessGroup;
 import org.apache.nifi.lifecycle.LifeCycleStartException;
 import org.apache.nifi.logging.LogLevel;
-import org.apache.nifi.nar.NarClassLoaders;
+import org.apache.nifi.nar.NarClassLoadersHolder;
 import org.apache.nifi.persistence.FlowConfigurationDAO;
 import org.apache.nifi.persistence.StandardXMLFlowConfigurationDAO;
 import org.apache.nifi.persistence.TemplateDeserializer;
@@ -190,7 +190,7 @@ public class StandardFlowService implements FlowService, ProtocolHandler {
         gracefulShutdownSeconds = (int) FormatUtils.getTimeDuration(nifiProperties.getProperty(NiFiProperties.FLOW_CONTROLLER_GRACEFUL_SHUTDOWN_PERIOD), TimeUnit.SECONDS);
         autoResumeState = nifiProperties.getAutoResumeState();
 
-        dao = new StandardXMLFlowConfigurationDAO(flowXml, encryptor, nifiProperties);
+        dao = new StandardXMLFlowConfigurationDAO(flowXml, encryptor, nifiProperties, controller.getExtensionManager());
         this.clusterCoordinator = clusterCoordinator;
         if (clusterCoordinator != null) {
             clusterCoordinator.setFlowService(this);
@@ -1057,7 +1057,7 @@ public class StandardFlowService implements FlowService, ProtocolHandler {
         public void run() {
             ClassLoader currentCl = null;
 
-            final Bundle frameworkBundle = NarClassLoaders.getInstance().getFrameworkBundle();
+            final Bundle frameworkBundle = NarClassLoadersHolder.getInstance().getFrameworkBundle();
             if (frameworkBundle != null) {
                 currentCl = Thread.currentThread().getContextClassLoader();
                 final ClassLoader cl = frameworkBundle.getClassLoader();

http://git-wip-us.apache.org/repos/asf/nifi/blob/fdd8cdbb/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardFlowSynchronizer.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardFlowSynchronizer.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardFlowSynchronizer.java
index d47e198..279daf3 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardFlowSynchronizer.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardFlowSynchronizer.java
@@ -58,6 +58,7 @@ import org.apache.nifi.groups.RemoteProcessGroup;
 import org.apache.nifi.groups.RemoteProcessGroupPortDescriptor;
 import org.apache.nifi.logging.ComponentLog;
 import org.apache.nifi.logging.LogLevel;
+import org.apache.nifi.nar.ExtensionManager;
 import org.apache.nifi.processor.Relationship;
 import org.apache.nifi.processor.SimpleProcessLogger;
 import org.apache.nifi.registry.flow.FlowRegistry;
@@ -137,11 +138,13 @@ public class StandardFlowSynchronizer implements FlowSynchronizer {
     private final StringEncryptor encryptor;
     private final boolean autoResumeState;
     private final NiFiProperties nifiProperties;
+    private final ExtensionManager extensionManager;
 
-    public StandardFlowSynchronizer(final StringEncryptor encryptor, final NiFiProperties nifiProperties) {
+    public StandardFlowSynchronizer(final StringEncryptor encryptor, final NiFiProperties nifiProperties, final ExtensionManager extensionManager) {
         this.encryptor = encryptor;
-        autoResumeState = nifiProperties.getAutoResumeState();
+        this.autoResumeState = nifiProperties.getAutoResumeState();
         this.nifiProperties = nifiProperties;
+        this.extensionManager = extensionManager;
     }
 
     public static boolean isEmpty(final DataFlow dataFlow) {
@@ -490,7 +493,7 @@ public class StandardFlowSynchronizer implements FlowSynchronizer {
                     if (!withinTemplate(componentElement)) {
                         final String componentType = DomUtils.getChildText(componentElement, "class");
                         try {
-                            BundleUtils.getBundle(componentType, FlowFromDOMFactory.getBundle(bundleElement));
+                            BundleUtils.getBundle(extensionManager, componentType, FlowFromDOMFactory.getBundle(bundleElement));
                         } catch (IllegalStateException e) {
                             throw new MissingBundleException(e.getMessage(), e);
                         }
@@ -644,7 +647,7 @@ public class StandardFlowSynchronizer implements FlowSynchronizer {
         if (!controllerInitialized || existingFlowEmpty) {
             BundleCoordinate coordinate;
             try {
-                coordinate = BundleUtils.getCompatibleBundle(dto.getType(), dto.getBundle());
+                coordinate = BundleUtils.getCompatibleBundle(extensionManager, dto.getType(), dto.getBundle());
             } catch (final IllegalStateException e) {
                 final BundleDTO bundleDTO = dto.getBundle();
                 if (bundleDTO == null) {
@@ -1222,7 +1225,7 @@ public class StandardFlowSynchronizer implements FlowSynchronizer {
 
             BundleCoordinate coordinate;
             try {
-                coordinate = BundleUtils.getCompatibleBundle(processorDTO.getType(), processorDTO.getBundle());
+                coordinate = BundleUtils.getCompatibleBundle(extensionManager, processorDTO.getType(), processorDTO.getBundle());
             } catch (final IllegalStateException e) {
                 final BundleDTO bundleDTO = processorDTO.getBundle();
                 if (bundleDTO == null) {
@@ -1651,7 +1654,7 @@ public class StandardFlowSynchronizer implements FlowSynchronizer {
         }
 
         // check if the Flow is inheritable
-        final FingerprintFactory fingerprintFactory = new FingerprintFactory(encryptor);
+        final FingerprintFactory fingerprintFactory = new FingerprintFactory(encryptor, extensionManager);
         final String existingFlowFingerprintBeforeHash = fingerprintFactory.createFingerprint(existingFlow, controller);
         if (existingFlowFingerprintBeforeHash.trim().isEmpty()) {
             return null;  // no existing flow, so equivalent to proposed flow

http://git-wip-us.apache.org/repos/asf/nifi/blob/fdd8cdbb/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 2cee3d4..8ab1d6f 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
@@ -82,6 +82,7 @@ import org.apache.nifi.groups.ProcessGroup;
 import org.apache.nifi.logging.ComponentLog;
 import org.apache.nifi.logging.LogLevel;
 import org.apache.nifi.logging.LogRepositoryFactory;
+import org.apache.nifi.nar.ExtensionManager;
 import org.apache.nifi.nar.NarCloseable;
 import org.apache.nifi.processor.ProcessContext;
 import org.apache.nifi.processor.ProcessSessionFactory;
@@ -152,20 +153,22 @@ 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 ComponentVariableRegistry variableRegistry, final ReloadComponent reloadComponent, final ValidationTrigger validationTrigger) {
+                                 final ComponentVariableRegistry variableRegistry, final ReloadComponent reloadComponent, final ExtensionManager extensionManager,
+                                 final ValidationTrigger validationTrigger) {
 
         this(processor, uuid, validationContextFactory, scheduler, controllerServiceProvider, processor.getComponent().getClass().getSimpleName(),
-            processor.getComponent().getClass().getCanonicalName(), nifiProperties, variableRegistry, reloadComponent, validationTrigger, false);
+            processor.getComponent().getClass().getCanonicalName(), nifiProperties, variableRegistry, reloadComponent, extensionManager, validationTrigger, 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 ComponentVariableRegistry variableRegistry, final ReloadComponent reloadComponent, final ValidationTrigger validationTrigger,
-                                 final boolean isExtensionMissing) {
+                                 final ComponentVariableRegistry variableRegistry, final ReloadComponent reloadComponent, final ExtensionManager extensionManager,
+                                 final ValidationTrigger validationTrigger, final boolean isExtensionMissing) {
 
-        super(uuid, validationContextFactory, controllerServiceProvider, componentType, componentCanonicalClass, variableRegistry, reloadComponent, validationTrigger, isExtensionMissing);
+        super(uuid, validationContextFactory, controllerServiceProvider, componentType, componentCanonicalClass, variableRegistry, reloadComponent,
+                extensionManager, validationTrigger, isExtensionMissing);
 
         final ProcessorDetails processorDetails = new ProcessorDetails(processor);
         this.processorRef = new AtomicReference<>(processorDetails);
@@ -916,7 +919,7 @@ public class StandardProcessorNode extends ProcessorNode implements Connectable
 
         final Set<Relationship> relationships;
         final Processor processor = processorRef.get().getProcessor();
-        try (final NarCloseable narCloseable = NarCloseable.withComponentNarLoader(processor.getClass(), processor.getIdentifier())) {
+        try (final NarCloseable narCloseable = NarCloseable.withComponentNarLoader(getExtensionManager(), processor.getClass(), processor.getIdentifier())) {
             relationships = processor.getRelationships();
         }
 
@@ -983,7 +986,7 @@ public class StandardProcessorNode extends ProcessorNode implements Connectable
         final Set<Relationship> undefined = new HashSet<>();
         final Set<Relationship> relationships;
         final Processor processor = processorRef.get().getProcessor();
-        try (final NarCloseable narCloseable = NarCloseable.withComponentNarLoader(processor.getClass(), processor.getIdentifier())) {
+        try (final NarCloseable narCloseable = NarCloseable.withComponentNarLoader(getExtensionManager(), processor.getClass(), processor.getIdentifier())) {
             relationships = processor.getRelationships();
         }
 
@@ -1131,7 +1134,7 @@ public class StandardProcessorNode extends ProcessorNode implements Connectable
     @Override
     public Collection<Relationship> getRelationships() {
         final Processor processor = processorRef.get().getProcessor();
-        try (final NarCloseable narCloseable = NarCloseable.withComponentNarLoader(processor.getClass(), processor.getIdentifier())) {
+        try (final NarCloseable narCloseable = NarCloseable.withComponentNarLoader(getExtensionManager(), processor.getClass(), processor.getIdentifier())) {
             return getProcessor().getRelationships();
         }
     }
@@ -1139,7 +1142,7 @@ public class StandardProcessorNode extends ProcessorNode implements Connectable
     @Override
     public String toString() {
         final Processor processor = processorRef.get().getProcessor();
-        try (final NarCloseable narCloseable = NarCloseable.withComponentNarLoader(processor.getClass(), processor.getIdentifier())) {
+        try (final NarCloseable narCloseable = NarCloseable.withComponentNarLoader(getExtensionManager(), processor.getClass(), processor.getIdentifier())) {
             return getProcessor().toString();
         }
     }
@@ -1161,7 +1164,7 @@ public class StandardProcessorNode extends ProcessorNode implements Connectable
         final Processor processor = processorRef.get().getProcessor();
 
         activateThread();
-        try (final NarCloseable narCloseable = NarCloseable.withComponentNarLoader(processor.getClass(), processor.getIdentifier())) {
+        try (final NarCloseable narCloseable = NarCloseable.withComponentNarLoader(getExtensionManager(), processor.getClass(), processor.getIdentifier())) {
             processor.onTrigger(context, sessionFactory);
         } finally {
             deactivateThread();
@@ -1497,7 +1500,7 @@ public class StandardProcessorNode extends ProcessorNode implements Connectable
             // Now that the task has been scheduled, set the timeout
             completionTimestampRef.set(System.currentTimeMillis() + onScheduleTimeoutMillis);
 
-            try (final NarCloseable nc = NarCloseable.withComponentNarLoader(processor.getClass(), processor.getIdentifier())) {
+            try (final NarCloseable nc = NarCloseable.withComponentNarLoader(getExtensionManager(), processor.getClass(), processor.getIdentifier())) {
                 try {
                     activateThread();
                     try {
@@ -1540,7 +1543,7 @@ public class StandardProcessorNode extends ProcessorNode implements Connectable
                     + "initialize and run the Processor again after the 'Administrative Yield Duration' has elapsed. Failure is due to " + e, e);
 
                 // If processor's task completed Exceptionally, then we want to retry initiating the start (if Processor is still scheduled to run).
-                try (final NarCloseable nc = NarCloseable.withComponentNarLoader(processor.getClass(), processor.getIdentifier())) {
+                try (final NarCloseable nc = NarCloseable.withComponentNarLoader(getExtensionManager(), processor.getClass(), processor.getIdentifier())) {
                     activateThread();
                     try {
                         ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnUnscheduled.class, processor, processContext);
@@ -1637,7 +1640,7 @@ public class StandardProcessorNode extends ProcessorNode implements Connectable
                             schedulingAgent.unschedule(StandardProcessorNode.this, scheduleState);
 
                             activateThread();
-                            try (final NarCloseable nc = NarCloseable.withComponentNarLoader(processor.getClass(), processor.getIdentifier())) {
+                            try (final NarCloseable nc = NarCloseable.withComponentNarLoader(getExtensionManager(), processor.getClass(), processor.getIdentifier())) {
                                 ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnUnscheduled.class, processor, processContext);
                             } finally {
                                 deactivateThread();
@@ -1649,7 +1652,7 @@ public class StandardProcessorNode extends ProcessorNode implements Connectable
                         final boolean allThreadsComplete = scheduleState.getActiveThreadCount() == 1;
                         if (allThreadsComplete) {
                             activateThread();
-                            try (final NarCloseable nc = NarCloseable.withComponentNarLoader(processor.getClass(), processor.getIdentifier())) {
+                            try (final NarCloseable nc = NarCloseable.withComponentNarLoader(getExtensionManager(), processor.getClass(), processor.getIdentifier())) {
                                 ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnStopped.class, processor, processContext);
                             } finally {
                                 deactivateThread();

http://git-wip-us.apache.org/repos/asf/nifi/blob/fdd8cdbb/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 9651add..bce85f8 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
@@ -40,6 +40,7 @@ import org.apache.nifi.controller.ValidationContextFactory;
 import org.apache.nifi.controller.service.ControllerServiceNode;
 import org.apache.nifi.controller.service.ControllerServiceProvider;
 import org.apache.nifi.controller.service.StandardConfigurationContext;
+import org.apache.nifi.nar.ExtensionManager;
 import org.apache.nifi.registry.ComponentVariableRegistry;
 import org.apache.nifi.reporting.ReportingTask;
 import org.apache.nifi.scheduling.SchedulingStrategy;
@@ -67,20 +68,22 @@ public abstract class AbstractReportingTaskNode extends AbstractComponentNode im
     public AbstractReportingTaskNode(final LoggableComponent<ReportingTask> reportingTask, final String id,
                                      final ControllerServiceProvider controllerServiceProvider, final ProcessScheduler processScheduler,
                                      final ValidationContextFactory validationContextFactory, final ComponentVariableRegistry variableRegistry,
-                                     final ReloadComponent reloadComponent, final ValidationTrigger validationTrigger) {
+                                     final ReloadComponent reloadComponent, final ExtensionManager extensionManager, final ValidationTrigger validationTrigger) {
 
         this(reportingTask, id, controllerServiceProvider, processScheduler, validationContextFactory,
                 reportingTask.getComponent().getClass().getSimpleName(), reportingTask.getComponent().getClass().getCanonicalName(),
-                variableRegistry, reloadComponent, validationTrigger, false);
+                variableRegistry, reloadComponent, extensionManager, validationTrigger, 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 ComponentVariableRegistry variableRegistry,
-                                     final ReloadComponent reloadComponent, final ValidationTrigger validationTrigger, final boolean isExtensionMissing) {
+                                     final ReloadComponent reloadComponent, final ExtensionManager extensionManager, final ValidationTrigger validationTrigger,
+                                     final boolean isExtensionMissing) {
 
-        super(id, validationContextFactory, controllerServiceProvider, componentType, componentCanonicalClass, variableRegistry, reloadComponent, validationTrigger, isExtensionMissing);
+        super(id, validationContextFactory, controllerServiceProvider, componentType, componentCanonicalClass, variableRegistry, reloadComponent,
+                extensionManager, validationTrigger, isExtensionMissing);
         this.reportingTaskRef = new AtomicReference<>(new ReportingTaskDetails(reportingTask));
         this.processScheduler = processScheduler;
         this.serviceLookup = controllerServiceProvider;

http://git-wip-us.apache.org/repos/asf/nifi/blob/fdd8cdbb/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 124f142..1cc5325 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
@@ -29,6 +29,7 @@ import org.apache.nifi.controller.ProcessScheduler;
 import org.apache.nifi.controller.ReloadComponent;
 import org.apache.nifi.controller.ReportingTaskNode;
 import org.apache.nifi.controller.ValidationContextFactory;
+import org.apache.nifi.nar.ExtensionManager;
 import org.apache.nifi.registry.ComponentVariableRegistry;
 import org.apache.nifi.reporting.ReportingContext;
 import org.apache.nifi.reporting.ReportingTask;
@@ -39,17 +40,19 @@ 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 ComponentVariableRegistry variableRegistry, final ReloadComponent reloadComponent, final ValidationTrigger validationTrigger) {
-        super(reportingTask, id, controller, processScheduler, validationContextFactory, variableRegistry, reloadComponent, validationTrigger);
+                                     final ComponentVariableRegistry variableRegistry, final ReloadComponent reloadComponent,
+                                     final ExtensionManager extensionManager, final ValidationTrigger validationTrigger) {
+        super(reportingTask, id, controller, processScheduler, validationContextFactory, variableRegistry, reloadComponent, extensionManager, validationTrigger);
         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 ComponentVariableRegistry variableRegistry,
-        final ReloadComponent reloadComponent, final ValidationTrigger validationTrigger, final boolean isExtensionMissing) {
+                                     final ReloadComponent reloadComponent, final ExtensionManager extensionManager, final ValidationTrigger validationTrigger,
+                                     final boolean isExtensionMissing) {
         super(reportingTask, id, controller, processScheduler, validationContextFactory, componentType, canonicalClassName,
-            variableRegistry, reloadComponent, validationTrigger, isExtensionMissing);
+            variableRegistry, reloadComponent, extensionManager, validationTrigger, isExtensionMissing);
         this.flowController = controller;
     }
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/fdd8cdbb/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/EventDrivenSchedulingAgent.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/EventDrivenSchedulingAgent.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/EventDrivenSchedulingAgent.java
index de97225..52dc89c 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/EventDrivenSchedulingAgent.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/EventDrivenSchedulingAgent.java
@@ -36,6 +36,7 @@ import org.apache.nifi.controller.service.ControllerServiceProvider;
 import org.apache.nifi.encrypt.StringEncryptor;
 import org.apache.nifi.engine.FlowEngine;
 import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.nar.ExtensionManager;
 import org.apache.nifi.nar.NarCloseable;
 import org.apache.nifi.processor.ProcessSessionFactory;
 import org.apache.nifi.processor.SimpleProcessLogger;
@@ -65,6 +66,7 @@ public class EventDrivenSchedulingAgent extends AbstractSchedulingAgent {
     private final AtomicInteger maxThreadCount;
     private final AtomicInteger activeThreadCount = new AtomicInteger(0);
     private final StringEncryptor encryptor;
+    private final ExtensionManager extensionManager;
 
     private volatile String adminYieldDuration = "1 sec";
 
@@ -72,7 +74,8 @@ public class EventDrivenSchedulingAgent extends AbstractSchedulingAgent {
     private final ConcurrentMap<Connectable, LifecycleState> scheduleStates = new ConcurrentHashMap<>();
 
     public EventDrivenSchedulingAgent(final FlowEngine flowEngine, final ControllerServiceProvider serviceProvider, final StateManagerProvider stateManagerProvider,
-        final EventDrivenWorkerQueue workerQueue, final RepositoryContextFactory contextFactory, final int maxThreadCount, final StringEncryptor encryptor) {
+                                      final EventDrivenWorkerQueue workerQueue, final RepositoryContextFactory contextFactory, final int maxThreadCount,
+                                      final StringEncryptor encryptor, final ExtensionManager extensionManager) {
         super(flowEngine);
         this.serviceProvider = serviceProvider;
         this.stateManagerProvider = stateManagerProvider;
@@ -80,6 +83,7 @@ public class EventDrivenSchedulingAgent extends AbstractSchedulingAgent {
         this.contextFactory = contextFactory;
         this.maxThreadCount = new AtomicInteger(maxThreadCount);
         this.encryptor = encryptor;
+        this.extensionManager = extensionManager;
 
         for (int i = 0; i < maxThreadCount; i++) {
             final Runnable eventDrivenTask = new EventDrivenTask(workerQueue, activeThreadCount);
@@ -305,7 +309,7 @@ public class EventDrivenSchedulingAgent extends AbstractSchedulingAgent {
             }
 
             try {
-                try (final AutoCloseable ncl = NarCloseable.withComponentNarLoader(worker.getClass(), worker.getIdentifier())) {
+                try (final AutoCloseable ncl = NarCloseable.withComponentNarLoader(extensionManager, worker.getClass(), worker.getIdentifier())) {
                     worker.onTrigger(processContext, sessionFactory);
                 } catch (final ProcessException pe) {
                     logger.error("{} failed to process session due to {}", worker, pe.toString());
@@ -323,7 +327,7 @@ public class EventDrivenSchedulingAgent extends AbstractSchedulingAgent {
                 }
             } finally {
                 if (!scheduleState.isScheduled() && scheduleState.getActiveThreadCount() == 1 && scheduleState.mustCallOnStoppedMethods()) {
-                    try (final NarCloseable x = NarCloseable.withComponentNarLoader(worker.getClass(), worker.getIdentifier())) {
+                    try (final NarCloseable x = NarCloseable.withComponentNarLoader(extensionManager, worker.getClass(), worker.getIdentifier())) {
                         ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnStopped.class, worker, processContext);
                     }
                 }
@@ -346,7 +350,7 @@ public class EventDrivenSchedulingAgent extends AbstractSchedulingAgent {
             }
 
             try {
-                try (final AutoCloseable ncl = NarCloseable.withComponentNarLoader(worker.getProcessor().getClass(), worker.getIdentifier())) {
+                try (final AutoCloseable ncl = NarCloseable.withComponentNarLoader(extensionManager, worker.getProcessor().getClass(), worker.getIdentifier())) {
                     worker.onTrigger(processContext, sessionFactory);
                 } catch (final ProcessException pe) {
                     final ComponentLog procLog = new SimpleProcessLogger(worker.getIdentifier(), worker.getProcessor());
@@ -365,7 +369,7 @@ public class EventDrivenSchedulingAgent extends AbstractSchedulingAgent {
                 // if the processor is no longer scheduled to run and this is the last thread,
                 // invoke the OnStopped methods
                 if (!scheduleState.isScheduled() && scheduleState.getActiveThreadCount() == 1 && scheduleState.mustCallOnStoppedMethods()) {
-                    try (final NarCloseable x = NarCloseable.withComponentNarLoader(worker.getProcessor().getClass(), worker.getIdentifier())) {
+                    try (final NarCloseable x = NarCloseable.withComponentNarLoader(extensionManager, worker.getProcessor().getClass(), worker.getIdentifier())) {
                         ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnStopped.class, worker.getProcessor(), processContext);
                     }
                 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/fdd8cdbb/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/QuartzSchedulingAgent.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/QuartzSchedulingAgent.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/QuartzSchedulingAgent.java
index 43c5e56..0f73c0e 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/QuartzSchedulingAgent.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/QuartzSchedulingAgent.java
@@ -74,7 +74,7 @@ public class QuartzSchedulingAgent extends AbstractSchedulingAgent {
             throw new IllegalStateException("Cannot schedule Reporting Task " + taskNode.getReportingTask().getIdentifier() + " to run because its scheduling period is not valid");
         }
 
-        final ReportingTaskWrapper taskWrapper = new ReportingTaskWrapper(taskNode, scheduleState);
+        final ReportingTaskWrapper taskWrapper = new ReportingTaskWrapper(taskNode, scheduleState, flowController.getExtensionManager());
 
         final AtomicBoolean canceled = new AtomicBoolean(false);
         final Date initialDate = cronExpression.getTimeAfter(new Date());

http://git-wip-us.apache.org/repos/asf/nifi/blob/fdd8cdbb/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/StandardProcessScheduler.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/StandardProcessScheduler.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/StandardProcessScheduler.java
index 0459372..4e396fd 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/StandardProcessScheduler.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/StandardProcessScheduler.java
@@ -210,7 +210,7 @@ public final class StandardProcessScheduler implements ProcessScheduler {
                             return;
                         }
 
-                        try (final NarCloseable x = NarCloseable.withComponentNarLoader(reportingTask.getClass(), reportingTask.getIdentifier())) {
+                        try (final NarCloseable x = NarCloseable.withComponentNarLoader(flowController.getExtensionManager(), reportingTask.getClass(), reportingTask.getIdentifier())) {
                             ReflectionUtils.invokeMethodsWithAnnotation(OnScheduled.class, reportingTask, taskNode.getConfigurationContext());
                         }
 
@@ -257,7 +257,7 @@ public final class StandardProcessScheduler implements ProcessScheduler {
                 synchronized (lifecycleState) {
                     lifecycleState.setScheduled(false);
 
-                    try (final NarCloseable x = NarCloseable.withComponentNarLoader(reportingTask.getClass(), reportingTask.getIdentifier())) {
+                    try (final NarCloseable x = NarCloseable.withComponentNarLoader(flowController.getExtensionManager(), reportingTask.getClass(), reportingTask.getIdentifier())) {
                         ReflectionUtils.invokeMethodsWithAnnotation(OnUnscheduled.class, reportingTask, configurationContext);
                     } catch (final Exception e) {
                         final Throwable cause = e instanceof InvocationTargetException ? e.getCause() : e;
@@ -483,7 +483,7 @@ public final class StandardProcessScheduler implements ProcessScheduler {
 
         if (!state.isScheduled() && state.getActiveThreadCount() == 0 && state.mustCallOnStoppedMethods()) {
             final ConnectableProcessContext processContext = new ConnectableProcessContext(connectable, encryptor, getStateManager(connectable.getIdentifier()));
-            try (final NarCloseable x = NarCloseable.withComponentNarLoader(connectable.getClass(), connectable.getIdentifier())) {
+            try (final NarCloseable x = NarCloseable.withComponentNarLoader(flowController.getExtensionManager(), connectable.getClass(), connectable.getIdentifier())) {
                 ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnStopped.class, connectable, processContext);
             }
         }

http://git-wip-us.apache.org/repos/asf/nifi/blob/fdd8cdbb/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/TimerDrivenSchedulingAgent.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/TimerDrivenSchedulingAgent.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/TimerDrivenSchedulingAgent.java
index 806fc67..db937e9 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/TimerDrivenSchedulingAgent.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/TimerDrivenSchedulingAgent.java
@@ -68,7 +68,7 @@ public class TimerDrivenSchedulingAgent extends AbstractSchedulingAgent {
 
     @Override
     public void doSchedule(final ReportingTaskNode taskNode, final LifecycleState scheduleState) {
-        final Runnable reportingTaskWrapper = new ReportingTaskWrapper(taskNode, scheduleState);
+        final Runnable reportingTaskWrapper = new ReportingTaskWrapper(taskNode, scheduleState, flowController.getExtensionManager());
         final long schedulingNanos = taskNode.getSchedulingPeriod(TimeUnit.NANOSECONDS);
 
         final ScheduledFuture<?> future = flowEngine.scheduleWithFixedDelay(reportingTaskWrapper, 0L, schedulingNanos, TimeUnit.NANOSECONDS);

http://git-wip-us.apache.org/repos/asf/nifi/blob/fdd8cdbb/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 e5192b3..e82c436 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
@@ -194,7 +194,7 @@ public class ControllerServiceLoader {
 
         BundleCoordinate coordinate;
         try {
-            coordinate = BundleUtils.getCompatibleBundle(dto.getType(), dto.getBundle());
+            coordinate = BundleUtils.getCompatibleBundle(provider.getExtensionManager(), dto.getType(), dto.getBundle());
         } catch (final IllegalStateException e) {
             final BundleDTO bundleDTO = dto.getBundle();
             if (bundleDTO == null) {

http://git-wip-us.apache.org/repos/asf/nifi/blob/fdd8cdbb/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceInvocationHandler.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceInvocationHandler.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceInvocationHandler.java
index ea83edc..1347e78 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceInvocationHandler.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceInvocationHandler.java
@@ -17,6 +17,7 @@
 package org.apache.nifi.controller.service;
 
 import org.apache.nifi.controller.ControllerService;
+import org.apache.nifi.nar.ExtensionManager;
 import org.apache.nifi.nar.NarCloseable;
 
 import java.lang.reflect.InvocationTargetException;
@@ -43,19 +44,21 @@ public class StandardControllerServiceInvocationHandler implements ControllerSer
 
     private final ControllerService originalService;
     private final AtomicReference<ControllerServiceNode> serviceNodeHolder = new AtomicReference<>(null);
+    private final ExtensionManager extensionManager;
 
     /**
      * @param originalService the original service being proxied
      */
-    public StandardControllerServiceInvocationHandler(final ControllerService originalService) {
-        this(originalService, null);
+    public StandardControllerServiceInvocationHandler(final ExtensionManager extensionManager, final ControllerService originalService) {
+        this(extensionManager, originalService, null);
     }
 
     /**
      * @param originalService the original service being proxied
      * @param serviceNode the node holding the original service which will be used for checking the state (disabled vs running)
      */
-    public StandardControllerServiceInvocationHandler(final ControllerService originalService, final ControllerServiceNode serviceNode) {
+    public StandardControllerServiceInvocationHandler(final ExtensionManager extensionManager, final ControllerService originalService, final ControllerServiceNode serviceNode) {
+        this.extensionManager = extensionManager;
         this.originalService = originalService;
         this.serviceNodeHolder.set(serviceNode);
     }
@@ -80,7 +83,7 @@ public class StandardControllerServiceInvocationHandler implements ControllerSer
                 + serviceNodeHolder.get().getIdentifier() + " because the Controller Service's State is currently " + state);
         }
 
-        try (final NarCloseable narCloseable = NarCloseable.withComponentNarLoader(originalService.getClass(), originalService.getIdentifier())) {
+        try (final NarCloseable narCloseable = NarCloseable.withComponentNarLoader(extensionManager, originalService.getClass(), originalService.getIdentifier())) {
             return method.invoke(originalService, args);
         } catch (final InvocationTargetException e) {
             // If the ControllerService throws an Exception, it'll be wrapped in an InvocationTargetException. We want

http://git-wip-us.apache.org/repos/asf/nifi/blob/fdd8cdbb/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 2323f02..795fc8c 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
@@ -61,6 +61,7 @@ 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.ExtensionManager;
 import org.apache.nifi.nar.NarCloseable;
 import org.apache.nifi.processor.SimpleProcessLogger;
 import org.apache.nifi.registry.ComponentVariableRegistry;
@@ -92,19 +93,19 @@ public class StandardControllerServiceNode extends AbstractComponentNode impleme
     public StandardControllerServiceNode(final LoggableComponent<ControllerService> implementation, final LoggableComponent<ControllerService> proxiedControllerService,
                                          final ControllerServiceInvocationHandler invocationHandler, final String id, final ValidationContextFactory validationContextFactory,
                                          final ControllerServiceProvider serviceProvider, final ComponentVariableRegistry variableRegistry, final ReloadComponent reloadComponent,
-                                         final ValidationTrigger validationTrigger) {
+                                         final ExtensionManager extensionManager, final ValidationTrigger validationTrigger) {
 
         this(implementation, proxiedControllerService, invocationHandler, id, validationContextFactory, serviceProvider, implementation.getComponent().getClass().getSimpleName(),
-            implementation.getComponent().getClass().getCanonicalName(), variableRegistry, reloadComponent, validationTrigger, false);
+            implementation.getComponent().getClass().getCanonicalName(), variableRegistry, reloadComponent, extensionManager, validationTrigger, 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 ComponentVariableRegistry variableRegistry, final ReloadComponent reloadComponent, final ValidationTrigger validationTrigger,
-                                         final boolean isExtensionMissing) {
+                                         final ComponentVariableRegistry variableRegistry, final ReloadComponent reloadComponent, final ExtensionManager extensionManager,
+                                         final ValidationTrigger validationTrigger, final boolean isExtensionMissing) {
 
-        super(id, validationContextFactory, serviceProvider, componentType, componentCanonicalClass, variableRegistry, reloadComponent, validationTrigger, isExtensionMissing);
+        super(id, validationContextFactory, serviceProvider, componentType, componentCanonicalClass, variableRegistry, reloadComponent, extensionManager, validationTrigger, isExtensionMissing);
         this.serviceProvider = serviceProvider;
         this.active = new AtomicBoolean();
         setControllerServiceAndProxy(implementation, proxiedControllerService, invocationHandler);
@@ -429,7 +430,7 @@ public class StandardControllerServiceNode extends AbstractComponentNode impleme
                 @Override
                 public void run() {
                     try {
-                        try (final NarCloseable nc = NarCloseable.withComponentNarLoader(getControllerServiceImplementation().getClass(), getIdentifier())) {
+                        try (final NarCloseable nc = NarCloseable.withComponentNarLoader(getExtensionManager(), getControllerServiceImplementation().getClass(), getIdentifier())) {
                             ReflectionUtils.invokeMethodsWithAnnotation(OnEnabled.class, getControllerServiceImplementation(), configContext);
                         }
 
@@ -459,7 +460,7 @@ public class StandardControllerServiceNode extends AbstractComponentNode impleme
                         if (isActive()) {
                             scheduler.schedule(this, administrativeYieldMillis, TimeUnit.MILLISECONDS);
                         } else {
-                            try (final NarCloseable nc = NarCloseable.withComponentNarLoader(getControllerServiceImplementation().getClass(), getIdentifier())) {
+                            try (final NarCloseable nc = NarCloseable.withComponentNarLoader(getExtensionManager(), getControllerServiceImplementation().getClass(), getIdentifier())) {
                                 ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnDisabled.class, getControllerServiceImplementation(), configContext);
                             }
                             stateTransition.disable();
@@ -529,7 +530,7 @@ public class StandardControllerServiceNode extends AbstractComponentNode impleme
 
 
     private void invokeDisable(ConfigurationContext configContext) {
-        try (final NarCloseable nc = NarCloseable.withComponentNarLoader(getControllerServiceImplementation().getClass(), getIdentifier())) {
+        try (final NarCloseable nc = NarCloseable.withComponentNarLoader(getExtensionManager(), getControllerServiceImplementation().getClass(), getIdentifier())) {
             ReflectionUtils.invokeMethodsWithAnnotation(OnDisabled.class, StandardControllerServiceNode.this.getControllerServiceImplementation(), configContext);
             LOG.debug("Successfully disabled {}", this);
         } catch (Exception e) {


[2/7] nifi git commit: NIFI-5673 Set up property/assembly for new auto-load directory - Set up NarAutoLoader to watch directory for new files - Move NarAutoLoader to JettyServer since it will need access to ExtensionManager - Created NarLoader to shared

Posted by ma...@apache.org.
http://git-wip-us.apache.org/repos/asf/nifi/blob/fdd8cdbb/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 8921b25..02fcaad 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
@@ -34,18 +34,20 @@ import java.util.LinkedHashMap;
 import java.util.LinkedHashSet;
 import java.util.List;
 import java.util.Map;
+import java.util.Optional;
 import java.util.Set;
 
 
 /**
- * A singleton class used to initialize the extension and framework classloaders.
+ * Used to initialize the extension and framework classloaders.
+ *
+ * The core framework should obtain a singleton reference from NarClassLoadersHolder.
  */
 public final class NarClassLoaders {
 
     public static final String FRAMEWORK_NAR_ID = "nifi-framework-nar";
     public static final String JETTY_NAR_ID = "nifi-jetty-bundle";
 
-    private static volatile NarClassLoaders ncl;
     private volatile InitContext initContext;
     private static final Logger logger = LoggerFactory.getLogger(NarClassLoaders.class);
 
@@ -54,39 +56,23 @@ public final class NarClassLoaders {
         private final File frameworkWorkingDir;
         private final File extensionWorkingDir;
         private final Bundle frameworkBundle;
+        private final Bundle jettyBundle;
         private final Map<String, Bundle> bundles;
 
         private InitContext(
                 final File frameworkDir,
                 final File extensionDir,
                 final Bundle frameworkBundle,
+                final Bundle jettyBundle,
                 final Map<String, Bundle> bundles) {
             this.frameworkWorkingDir = frameworkDir;
             this.extensionWorkingDir = extensionDir;
             this.frameworkBundle = frameworkBundle;
+            this.jettyBundle = jettyBundle;
             this.bundles = bundles;
         }
     }
 
-    private NarClassLoaders() {
-    }
-
-    /**
-     * @return The singleton instance of the NarClassLoaders
-     */
-    public static NarClassLoaders getInstance() {
-        NarClassLoaders result = ncl;
-        if (result == null) {
-            synchronized (NarClassLoaders.class) {
-                result = ncl;
-                if (result == null) {
-                    ncl = result = new NarClassLoaders();
-                }
-            }
-        }
-        return result;
-    }
-
     /**
      * Initializes and loads the NarClassLoaders. This method must be called
      * before the rest of the methods to access the classloaders are called and
@@ -177,6 +163,7 @@ public final class NarClassLoaders {
                 } catch (IllegalStateException e) {
                     logger.warn("Unable to load NAR {} due to {}, skipping...",
                             new Object[] {unpackedNar.getAbsolutePath(), e.getMessage()});
+                    continue;
                 }
 
                 // prevent the application from starting when there are two NARs with same group, id, and version
@@ -286,7 +273,159 @@ public final class NarClassLoaders {
                 .filter(b -> b.getBundleDetails().getCoordinate().getId().equals(FRAMEWORK_NAR_ID))
                 .findFirst().orElse(null);
 
-        return new InitContext(frameworkWorkingDir, extensionsWorkingDir, frameworkBundle, new LinkedHashMap<>(narDirectoryBundleLookup));
+        // find the Jetty bundle
+        final Bundle jettyBundle = narDirectoryBundleLookup.values().stream()
+                .filter(b -> b.getBundleDetails().getCoordinate().getId().equals(JETTY_NAR_ID))
+                .findFirst().orElse(null);
+
+        if (jettyBundle == null) {
+            throw new IllegalStateException("Unable to locate Jetty bundle.");
+        }
+
+        return new InitContext(frameworkWorkingDir, extensionsWorkingDir, frameworkBundle, jettyBundle, new LinkedHashMap<>(narDirectoryBundleLookup));
+    }
+
+    /**
+     * Loads additional NARs after the application has been started.
+     *
+     * @param additionalUnpackedNars a list of files where each file represents a directory of an unpacked NAR to load
+     * @return the result which includes the loaded bundles and details of skipped bundles
+     */
+    public synchronized NarLoadResult loadAdditionalNars(final List<File> additionalUnpackedNars) {
+        if (initContext == null) {
+            throw new IllegalStateException("Must call init before attempting to load additional NARs");
+        }
+
+        final Set<Bundle> loadedBundles = new LinkedHashSet<>();
+        final List<BundleDetails> additionalBundleDetails = loadBundleDetails(additionalUnpackedNars);
+
+        // Create a lookup from bundle id to set of coordinates with that id, needs to be across already loaded NARs + additional NARs currently being loaded
+        final Map<String,Set<BundleCoordinate>> bundleIdToCoordinatesLookup = new HashMap<>();
+
+        // Add the coordinates from the additional bundles
+        for (final BundleDetails bundleDetail : additionalBundleDetails) {
+            final String bundleId = bundleDetail.getCoordinate().getId();
+            final Set<BundleCoordinate> coordinates = bundleIdToCoordinatesLookup.computeIfAbsent(bundleId, (id) -> new HashSet<>());
+            coordinates.add(bundleDetail.getCoordinate());
+        }
+
+        // Add coordinates from the already loaded bundles
+        for (final Bundle bundle : getBundles()) {
+            final BundleDetails bundleDetail = bundle.getBundleDetails();
+            final String bundleId = bundleDetail.getCoordinate().getId();
+            final Set<BundleCoordinate> coordinates = bundleIdToCoordinatesLookup.computeIfAbsent(bundleId, (id) -> new HashSet<>());
+            coordinates.add(bundleDetail.getCoordinate());
+        }
+
+        int bundleCount;
+        do {
+            // Record the number of bundles to be loaded
+            bundleCount = additionalBundleDetails.size();
+
+            // Attempt to create each bundle class loader
+            for (final Iterator<BundleDetails> additionalBundleDetailsIter = additionalBundleDetails.iterator(); additionalBundleDetailsIter.hasNext();) {
+                final BundleDetails bundleDetail = additionalBundleDetailsIter.next();
+                try {
+                    // If we were able to create the bundle class loader, store it and remove the details
+                    final ClassLoader bundleClassLoader = createBundleClassLoader(bundleDetail, bundleIdToCoordinatesLookup);
+                    if (bundleClassLoader != null) {
+                        final Bundle bundle = new Bundle(bundleDetail, bundleClassLoader);
+                        loadedBundles.add(bundle);
+                        additionalBundleDetailsIter.remove();
+
+                        // Need to add to overall bundles as we go so if other NARs depend on this one we can find it
+                        initContext.bundles.put(bundleDetail.getWorkingDirectory().getCanonicalPath(), bundle);
+                    }
+                } catch (final Exception e) {
+                    logger.error("Unable to load NAR {} due to {}, skipping...", new Object[]{bundleDetail.getWorkingDirectory(), e.getMessage()});
+                }
+            }
+
+            // Attempt to load more if some were successfully loaded this iteration
+        } while (bundleCount != additionalBundleDetails.size());
+
+        // See if any bundles couldn't be loaded
+        final Set<BundleDetails> skippedBundles = new HashSet<>();
+        for (final BundleDetails bundleDetail : additionalBundleDetails) {
+            logger.warn(String.format("Unable to resolve required dependency '%s'. Skipping NAR '%s'",
+                    bundleDetail.getDependencyCoordinate().getId(), bundleDetail.getWorkingDirectory().getAbsolutePath()));
+            skippedBundles.add(bundleDetail);
+        }
+
+        return new NarLoadResult(loadedBundles, skippedBundles);
+    }
+
+    private ClassLoader createBundleClassLoader(final BundleDetails bundleDetail, final Map<String,Set<BundleCoordinate>> bundleIdToCoordinatesLookup)
+            throws IOException, ClassNotFoundException {
+
+        ClassLoader bundleClassLoader = null;
+
+        final BundleCoordinate bundleDependencyCoordinate = bundleDetail.getDependencyCoordinate();
+        if (bundleDependencyCoordinate == null) {
+            final ClassLoader jettyClassLoader = getJettyBundle().getClassLoader();
+            bundleClassLoader = createNarClassLoader(bundleDetail.getWorkingDirectory(), jettyClassLoader);
+        } else {
+            final Optional<Bundle> dependencyBundle = getBundle(bundleDependencyCoordinate);
+
+            // If the declared dependency has already been loaded then use it
+            if (dependencyBundle.isPresent()) {
+                final ClassLoader narDependencyClassLoader = dependencyBundle.get().getClassLoader();
+                bundleClassLoader = createNarClassLoader(bundleDetail.getWorkingDirectory(), narDependencyClassLoader);
+            } else {
+                // Otherwise get all bundles that match the declared dependency id
+                final Set<BundleCoordinate> coordinates = bundleIdToCoordinatesLookup.get(bundleDependencyCoordinate.getId());
+
+                // Ensure there are known bundles that match the declared dependency id
+                if (coordinates != null && !coordinates.contains(bundleDependencyCoordinate)) {
+                    // Ensure the declared dependency only has one possible bundle
+                    if (coordinates.size() == 1) {
+                        // Get the bundle with the matching id
+                        final BundleCoordinate coordinate = coordinates.stream().findFirst().get();
+
+                        // If that bundle is loaded, use it
+                        final Optional<Bundle> matchingDependencyIdBundle = getBundle(coordinate);
+                        if (matchingDependencyIdBundle.isPresent()) {
+                            final String dependencyCoordinateStr = bundleDependencyCoordinate.getCoordinate();
+                            logger.warn(String.format("While loading '%s' unable to locate exact NAR dependency '%s'. Only found one possible match '%s'. Continuing...",
+                                    bundleDetail.getCoordinate().getCoordinate(), dependencyCoordinateStr, coordinate.getCoordinate()));
+
+                            final ClassLoader narDependencyClassLoader = matchingDependencyIdBundle.get().getClassLoader();
+                            bundleClassLoader = createNarClassLoader(bundleDetail.getWorkingDirectory(), narDependencyClassLoader);
+                        }
+                    }
+                }
+            }
+        }
+
+        return bundleClassLoader;
+    }
+
+    private List<BundleDetails> loadBundleDetails(List<File> unpackedNars) {
+        final List<BundleDetails> narDetails = new ArrayList<>();
+        for (final File unpackedNar : unpackedNars) {
+            try {
+                final BundleDetails narDetail = getNarDetails(unpackedNar);
+                final BundleCoordinate unpackedNarCoordinate = narDetail.getCoordinate();
+
+                // Skip this NAR if there is another NAR with the same group, id, and version
+                final Optional<Bundle> existingBundle = getBundle(unpackedNarCoordinate);
+                if (existingBundle.isPresent()) {
+                    final BundleDetails existingBundleDetails = existingBundle.get().getBundleDetails();
+                    final String existingNarWorkingDir = existingBundleDetails.getWorkingDirectory().getCanonicalPath();
+                    final String unpackedNarWorkingDir = narDetail.getWorkingDirectory().getCanonicalPath();
+
+                    logger.error("Unable to load NAR with coordinates {} and working directory {} " +
+                                    "because another NAR with the same coordinates already exists at {}",
+                            new Object[]{unpackedNarCoordinate, unpackedNarWorkingDir, existingNarWorkingDir});
+                } else {
+                    narDetails.add(narDetail);
+                }
+
+            } catch (Exception e) {
+                logger.error("Unable to load NAR {} due to {}, skipping...", new Object[]{unpackedNar.getAbsolutePath(), e.getMessage()});
+            }
+        }
+        return narDetails;
     }
 
     /**
@@ -318,6 +457,18 @@ public final class NarClassLoaders {
     }
 
     /**
+     * Gets the bundle with the given coordinate.
+     *
+     * @param bundleCoordinate the coordinate of the bundle to find
+     * @return the bundle with the coordinate, or an empty optional
+     */
+    private Optional<Bundle> getBundle(final BundleCoordinate bundleCoordinate) {
+        return initContext.bundles.values().stream()
+                .filter(b -> b.getBundleDetails().getCoordinate().equals(bundleCoordinate))
+                .findFirst();
+    }
+
+    /**
      * @return the framework class Bundle
      *
      * @throws IllegalStateException if the frame Bundle has not been loaded
@@ -331,6 +482,19 @@ public final class NarClassLoaders {
     }
 
     /**
+     * @return the Jetty Bundle
+     *
+     * @throws IllegalStateException if the Jetty Bundle has not been loaded
+     */
+    public Bundle getJettyBundle() {
+        if (initContext == null) {
+            throw new IllegalStateException("Jetty bundle has not been loaded.");
+        }
+
+        return initContext.jettyBundle;
+    }
+
+    /**
      * @param extensionWorkingDirectory the directory
      * @return the bundle for the specified working directory. Returns
      * null when no bundle exists for the specified working directory

http://git-wip-us.apache.org/repos/asf/nifi/blob/fdd8cdbb/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/nar/NarClassLoadersHolder.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/nar/NarClassLoadersHolder.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/nar/NarClassLoadersHolder.java
new file mode 100644
index 0000000..692dfcb
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/nar/NarClassLoadersHolder.java
@@ -0,0 +1,41 @@
+/*
+ * 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;
+
+public final class NarClassLoadersHolder {
+
+    private static volatile NarClassLoaders INSTANCE;
+
+    /**
+     * @return The singleton instance of the NarClassLoaders
+     */
+    public static NarClassLoaders getInstance() {
+        if (INSTANCE == null) {
+            synchronized (NarClassLoadersHolder.class) {
+                if (INSTANCE == null) {
+                    INSTANCE = new NarClassLoaders();
+                }
+            }
+        }
+        return INSTANCE;
+    }
+
+    private NarClassLoadersHolder() {
+
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/fdd8cdbb/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/nar/NarLoadResult.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/nar/NarLoadResult.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/nar/NarLoadResult.java
new file mode 100644
index 0000000..652893f
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/nar/NarLoadResult.java
@@ -0,0 +1,46 @@
+/*
+ * 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.BundleDetails;
+
+import java.util.Set;
+
+/**
+ * Holder so we can return which bundles were loaded and which were skipped.
+ */
+public class NarLoadResult {
+
+    private final Set<Bundle> loadedBundles;
+
+    private final Set<BundleDetails> skippedBundles;
+
+    public NarLoadResult(final Set<Bundle> loadedBundles, final Set<BundleDetails> skippedBundles) {
+        this.loadedBundles = loadedBundles;
+        this.skippedBundles = skippedBundles;
+    }
+
+    public Set<Bundle> getLoadedBundles() {
+        return loadedBundles;
+    }
+
+    public Set<BundleDetails> getSkippedBundles() {
+        return skippedBundles;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/fdd8cdbb/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/nar/NarUnpacker.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/nar/NarUnpacker.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/nar/NarUnpacker.java
index afd021e..edb824f 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/nar/NarUnpacker.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/nar/NarUnpacker.java
@@ -201,6 +201,11 @@ public final class NarUnpacker {
         }
     }
 
+    public static void mapExtension(final File unpackedNar, final BundleCoordinate bundleCoordinate, final File docsDirectory, final ExtensionMapping mapping) throws IOException {
+        final File bundledDependencies = new File(unpackedNar, "NAR-INF/bundled-dependencies");
+        unpackBundleDocs(docsDirectory, mapping, bundleCoordinate, bundledDependencies);
+    }
+
     private static void unpackBundleDocs(final File docsDirectory, final ExtensionMapping mapping, final BundleCoordinate bundleCoordinate, final File bundledDirectory) throws IOException {
         final File[] directoryContents = bundledDirectory.listFiles();
         if (directoryContents != null) {
@@ -220,7 +225,7 @@ public final class NarUnpacker {
      * @return the directory to the unpacked NAR
      * @throws IOException if unable to explode nar
      */
-    private static File unpackNar(final File nar, final File baseWorkingDirectory) throws IOException {
+    public static File unpackNar(final File nar, final File baseWorkingDirectory) throws IOException {
         final File narWorkingDirectory = new File(baseWorkingDirectory, nar.getName() + "-unpacked");
 
         // if the working directory doesn't exist, unpack the nar

http://git-wip-us.apache.org/repos/asf/nifi/blob/fdd8cdbb/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/pom.xml
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/pom.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/pom.xml
index 0c5d1a8..5379e6c 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/pom.xml
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/pom.xml
@@ -80,6 +80,7 @@
         <nifi.ui.banner.text />
         <nifi.ui.autorefresh.interval>30 sec</nifi.ui.autorefresh.interval>
         <nifi.nar.library.directory>./lib</nifi.nar.library.directory>
+        <nifi.nar.library.autoload.directory>./extensions</nifi.nar.library.autoload.directory>
         <nifi.nar.working.directory>./work/nar/</nifi.nar.working.directory>
         <nifi.documentation.working.directory>./work/docs/components</nifi.documentation.working.directory>
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/fdd8cdbb/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/nifi.properties
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/nifi.properties b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/nifi.properties
index 6143f63..8e8814e 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/nifi.properties
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/nifi.properties
@@ -35,6 +35,7 @@ nifi.templates.directory=${nifi.templates.directory}
 nifi.ui.banner.text=${nifi.ui.banner.text}
 nifi.ui.autorefresh.interval=${nifi.ui.autorefresh.interval}
 nifi.nar.library.directory=${nifi.nar.library.directory}
+nifi.nar.library.autoload.directory=${nifi.nar.library.autoload.directory}
 nifi.nar.working.directory=${nifi.nar.working.directory}
 nifi.documentation.working.directory=${nifi.documentation.working.directory}
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/fdd8cdbb/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 13b60e8..7a44250 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
@@ -19,6 +19,7 @@ package org.apache.nifi;
 import org.apache.nifi.bundle.Bundle;
 import org.apache.nifi.nar.ExtensionMapping;
 import org.apache.nifi.nar.NarClassLoaders;
+import org.apache.nifi.nar.NarClassLoadersHolder;
 import org.apache.nifi.nar.NarUnpacker;
 import org.apache.nifi.nar.SystemBundle;
 import org.apache.nifi.util.FileUtils;
@@ -128,7 +129,7 @@ public class NiFi {
         final ExtensionMapping extensionMapping = NarUnpacker.unpackNars(properties, systemBundle);
 
         // load the extensions classloaders
-        NarClassLoaders narClassLoaders = NarClassLoaders.getInstance();
+        NarClassLoaders narClassLoaders = NarClassLoadersHolder.getInstance();
 
         narClassLoaders.init(rootClassLoader,
                 properties.getFrameworkWorkingDirectory(), properties.getExtensionsWorkingDirectory());

http://git-wip-us.apache.org/repos/asf/nifi/blob/fdd8cdbb/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 ce4aeae..023780e 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
@@ -55,7 +55,7 @@
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
-            <artifactId>nifi-documentation</artifactId>
+            <artifactId>nifi-framework-nar-loading-utils</artifactId>
             <scope>compile</scope>
         </dependency>
         <dependency>
@@ -87,6 +87,10 @@
         </dependency>
         <dependency>
             <groupId>org.eclipse.jetty</groupId>
+            <artifactId>jetty-deploy</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.eclipse.jetty</groupId>
             <artifactId>jetty-annotations</artifactId>
         </dependency>
         <dependency>

http://git-wip-us.apache.org/repos/asf/nifi/blob/fdd8cdbb/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 3bf98ca..401d8e9 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
@@ -18,40 +18,6 @@ package org.apache.nifi.web.server;
 
 import com.google.common.base.Strings;
 import com.google.common.collect.Lists;
-import java.io.BufferedReader;
-import java.io.File;
-import java.io.FileFilter;
-import java.io.IOException;
-import java.io.InputStreamReader;
-import java.net.InetAddress;
-import java.net.NetworkInterface;
-import java.net.SocketException;
-import java.nio.file.Paths;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.EnumSet;
-import java.util.Enumeration;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Objects;
-import java.util.Set;
-import java.util.concurrent.TimeUnit;
-import java.util.jar.JarEntry;
-import java.util.jar.JarFile;
-import java.util.stream.Collectors;
-import javax.servlet.DispatcherType;
-import javax.servlet.Filter;
-import javax.servlet.FilterChain;
-import javax.servlet.FilterConfig;
-import javax.servlet.ServletContext;
-import javax.servlet.ServletException;
-import javax.servlet.ServletRequest;
-import javax.servlet.ServletResponse;
-import javax.servlet.http.HttpServletResponse;
 import org.apache.commons.collections4.CollectionUtils;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.nifi.NiFiServer;
@@ -62,8 +28,15 @@ 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.ExtensionDiscoveringManager;
+import org.apache.nifi.nar.ExtensionManagerHolder;
 import org.apache.nifi.nar.ExtensionMapping;
+import org.apache.nifi.nar.ExtensionUiLoader;
+import org.apache.nifi.nar.NarAutoLoader;
+import org.apache.nifi.nar.NarClassLoadersHolder;
+import org.apache.nifi.nar.NarLoader;
+import org.apache.nifi.nar.StandardExtensionDiscoveringManager;
+import org.apache.nifi.nar.StandardNarLoader;
 import org.apache.nifi.processor.DataUnit;
 import org.apache.nifi.security.util.KeyStoreUtils;
 import org.apache.nifi.services.FlowService;
@@ -76,6 +49,8 @@ import org.apache.nifi.web.NiFiWebConfigurationContext;
 import org.apache.nifi.web.UiExtensionType;
 import org.apache.nifi.web.security.ContentSecurityPolicyFilter;
 import org.eclipse.jetty.annotations.AnnotationConfiguration;
+import org.eclipse.jetty.deploy.App;
+import org.eclipse.jetty.deploy.DeploymentManager;
 import org.eclipse.jetty.server.Connector;
 import org.eclipse.jetty.server.Handler;
 import org.eclipse.jetty.server.HttpConfiguration;
@@ -84,6 +59,7 @@ import org.eclipse.jetty.server.SecureRequestCustomizer;
 import org.eclipse.jetty.server.Server;
 import org.eclipse.jetty.server.ServerConnector;
 import org.eclipse.jetty.server.SslConnectionFactory;
+import org.eclipse.jetty.server.handler.ContextHandlerCollection;
 import org.eclipse.jetty.server.handler.HandlerCollection;
 import org.eclipse.jetty.server.handler.HandlerList;
 import org.eclipse.jetty.server.handler.gzip.GzipHandler;
@@ -103,14 +79,52 @@ import org.springframework.context.ApplicationContext;
 import org.springframework.web.context.WebApplicationContext;
 import org.springframework.web.context.support.WebApplicationContextUtils;
 
+import javax.servlet.DispatcherType;
+import javax.servlet.Filter;
+import javax.servlet.FilterChain;
+import javax.servlet.FilterConfig;
+import javax.servlet.ServletContext;
+import javax.servlet.ServletException;
+import javax.servlet.ServletRequest;
+import javax.servlet.ServletResponse;
+import javax.servlet.http.HttpServletResponse;
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileFilter;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.net.InetAddress;
+import java.net.NetworkInterface;
+import java.net.SocketException;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.EnumSet;
+import java.util.Enumeration;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.jar.JarEntry;
+import java.util.jar.JarFile;
+import java.util.stream.Collectors;
+
 /**
  * Encapsulates the Jetty instance.
  */
-public class JettyServer implements NiFiServer {
+public class JettyServer implements NiFiServer, ExtensionUiLoader {
 
     private static final Logger logger = LoggerFactory.getLogger(JettyServer.class);
     private static final String WEB_DEFAULTS_XML = "org/apache/nifi/web/webdefault.xml";
 
+    private static final String CONTAINER_INCLUDE_PATTERN_KEY = "org.eclipse.jetty.server.webapp.ContainerIncludeJarPattern";
+    private static final String CONTAINER_INCLUDE_PATTERN_VALUE = ".*/[^/]*servlet-api-[^/]*\\.jar$|.*/javax.servlet.jsp.jstl-.*\\\\.jar$|.*/[^/]*taglibs.*\\.jar$";
+
     private static final FileFilter WAR_FILTER = new FileFilter() {
         @Override
         public boolean accept(File pathname) {
@@ -125,6 +139,7 @@ public class JettyServer implements NiFiServer {
     private Bundle systemBundle;
     private Set<Bundle> bundles;
     private ExtensionMapping extensionMapping;
+    private NarAutoLoader narAutoLoader;
 
     private WebAppContext webApiContext;
     private WebAppContext webDocsContext;
@@ -137,6 +152,8 @@ public class JettyServer implements NiFiServer {
     private UiExtensionMapping componentUiExtensions;
     private Collection<WebAppContext> componentUiExtensionWebContexts;
 
+    private DeploymentManager deploymentManager;
+
     public JettyServer(final NiFiProperties props, final Set<Bundle> bundles) {
         final QueuedThreadPool threadPool = new QueuedThreadPool(props.getWebThreads());
         threadPool.setName("NiFi Web Server");
@@ -153,9 +170,9 @@ public class JettyServer implements NiFiServer {
         configureConnectors(server);
 
         // load wars from the bundle
-        Handler warHandlers = loadWars(bundles);
+        final Handler warHandlers = loadInitialWars(bundles);
 
-        HandlerList allHandlers = new HandlerList();
+        final HandlerList allHandlers = new HandlerList();
 
         // Only restrict the host header if running in HTTPS mode
         if (props.isHTTPSConfigured()) {
@@ -169,8 +186,16 @@ public class JettyServer implements NiFiServer {
             logger.info("Running in HTTP mode; host headers not restricted");
         }
 
-        allHandlers.addHandler(warHandlers);
+
+        final ContextHandlerCollection contextHandlers = new ContextHandlerCollection();
+        contextHandlers.addHandler(warHandlers);
+        allHandlers.addHandler(contextHandlers);
         server.setHandler(allHandlers);
+
+        deploymentManager = new DeploymentManager();
+        deploymentManager.setContextAttribute(CONTAINER_INCLUDE_PATTERN_KEY, CONTAINER_INCLUDE_PATTERN_VALUE);
+        deploymentManager.setContexts(contextHandlers);
+        server.addBean(deploymentManager);
     }
 
     /**
@@ -181,7 +206,7 @@ public class JettyServer implements NiFiServer {
         this.props = properties;
     }
 
-    private Handler loadWars(final Set<Bundle> bundles) {
+    private Handler loadInitialWars(final Set<Bundle> bundles) {
 
         // load WARs
         final Map<File, Bundle> warToBundleLookup = findWars(bundles);
@@ -192,8 +217,11 @@ public class JettyServer implements NiFiServer {
         File webErrorWar = null;
         File webDocsWar = null;
         File webContentViewerWar = null;
-        List<File> otherWars = new ArrayList<>();
-        for (File war : warToBundleLookup.keySet()) {
+        Map<File, Bundle> otherWars = new HashMap<>();
+        for (Map.Entry<File,Bundle> warBundleEntry : warToBundleLookup.entrySet()) {
+            final File war = warBundleEntry.getKey();
+            final Bundle warBundle = warBundleEntry.getValue();
+
             if (war.getName().toLowerCase().startsWith("nifi-web-api")) {
                 webApiWar = war;
             } else if (war.getName().toLowerCase().startsWith("nifi-web-error")) {
@@ -205,7 +233,7 @@ public class JettyServer implements NiFiServer {
             } else if (war.getName().toLowerCase().startsWith("nifi-web")) {
                 webUiWar = war;
             } else {
-                otherWars.add(war);
+                otherWars.put(war, warBundle);
             }
         }
 
@@ -223,20 +251,114 @@ public class JettyServer implements NiFiServer {
         }
 
         // handlers for each war and init params for the web api
-        final HandlerCollection handlers = new HandlerCollection();
+        final ExtensionUiInfo extensionUiInfo = loadWars(otherWars);
+        componentUiExtensionWebContexts = new ArrayList<>(extensionUiInfo.getComponentUiExtensionWebContexts());
+        contentViewerWebContexts = new ArrayList<>(extensionUiInfo.getContentViewerWebContexts());
+        componentUiExtensions = new UiExtensionMapping(extensionUiInfo.getComponentUiExtensionsByType());
+
+        final HandlerCollection webAppContextHandlers = new HandlerCollection();
+        final Collection<WebAppContext> extensionUiContexts = extensionUiInfo.getWebAppContexts();
+        extensionUiContexts.stream().forEach(c -> webAppContextHandlers.addHandler(c));
+
+        final ClassLoader frameworkClassLoader = getClass().getClassLoader();
+
+        // load the web ui app
+        final WebAppContext webUiContext = loadWar(webUiWar, "/nifi", frameworkClassLoader);
+        webUiContext.getInitParams().put("oidc-supported", String.valueOf(props.isOidcEnabled()));
+        webUiContext.getInitParams().put("knox-supported", String.valueOf(props.isKnoxSsoEnabled()));
+        webUiContext.getInitParams().put("whitelistedContextPaths", props.getWhitelistedContextPaths());
+        webAppContextHandlers.addHandler(webUiContext);
+
+        // load the web api app
+        webApiContext = loadWar(webApiWar, "/nifi-api", frameworkClassLoader);
+        webAppContextHandlers.addHandler(webApiContext);
+
+        // load the content viewer app
+        webContentViewerContext = loadWar(webContentViewerWar, "/nifi-content-viewer", frameworkClassLoader);
+        webContentViewerContext.getInitParams().putAll(extensionUiInfo.getMimeMappings());
+        webAppContextHandlers.addHandler(webContentViewerContext);
+
+        // create a web app for the docs
+        final String docsContextPath = "/nifi-docs";
+
+        // load the documentation war
+        webDocsContext = loadWar(webDocsWar, docsContextPath, frameworkClassLoader);
+
+        // add the servlets which serve the HTML documentation within the documentation web app
+        addDocsServlets(webDocsContext);
+
+        webAppContextHandlers.addHandler(webDocsContext);
+
+        // load the web error app
+        final WebAppContext webErrorContext = loadWar(webErrorWar, "/", frameworkClassLoader);
+        webErrorContext.getInitParams().put("whitelistedContextPaths", props.getWhitelistedContextPaths());
+        webAppContextHandlers.addHandler(webErrorContext);
+
+        // deploy the web apps
+        return gzip(webAppContextHandlers);
+    }
+
+    @Override
+    public void loadExtensionUis(final Set<Bundle> bundles) {
+         // Find and load any WARs contained within the set of bundles...
+        final Map<File, Bundle> warToBundleLookup = findWars(bundles);
+        final ExtensionUiInfo extensionUiInfo = loadWars(warToBundleLookup);
+
+        final Collection<WebAppContext> webAppContexts = extensionUiInfo.getWebAppContexts();
+        if (CollectionUtils.isEmpty(webAppContexts)) {
+            logger.debug("No webapp contexts were loaded, returning...");
+            return;
+        }
+
+        // Deploy each WAR that was loaded...
+        for (final WebAppContext webAppContext : webAppContexts) {
+            final App extensionUiApp = new App(deploymentManager, null, "nifi-jetty-server", webAppContext);
+            deploymentManager.addApp(extensionUiApp);
+        }
+
+        final Collection<WebAppContext> componentUiExtensionWebContexts = extensionUiInfo.getComponentUiExtensionWebContexts();
+        final Collection<WebAppContext> contentViewerWebContexts = extensionUiInfo.getContentViewerWebContexts();
+
+        // Inject the configuration context and security filter into contexts that need it
+        final ServletContext webApiServletContext = webApiContext.getServletHandler().getServletContext();
+        final WebApplicationContext webApplicationContext = WebApplicationContextUtils.getRequiredWebApplicationContext(webApiServletContext);
+        final NiFiWebConfigurationContext configurationContext = webApplicationContext.getBean("nifiWebConfigurationContext", NiFiWebConfigurationContext.class);
+        final FilterHolder securityFilter = webApiContext.getServletHandler().getFilter("springSecurityFilterChain");
+
+        performInjectionForComponentUis(componentUiExtensionWebContexts, configurationContext, securityFilter);
+        performInjectionForContentViewerUis(contentViewerWebContexts, securityFilter);
+
+        // Merge results of current loading into previously loaded results...
+        this.componentUiExtensionWebContexts.addAll(componentUiExtensionWebContexts);
+        this.contentViewerWebContexts.addAll(contentViewerWebContexts);
+        this.componentUiExtensions.addUiExtensions(extensionUiInfo.getComponentUiExtensionsByType());
+
+        for (final WebAppContext webAppContext : webAppContexts) {
+            final Throwable t = webAppContext.getUnavailableException();
+            if (t != null) {
+                logger.error("Unable to start context due to " + t.getMessage(), t);
+            }
+        }
+    }
+
+    private ExtensionUiInfo loadWars(final Map<File, Bundle> warToBundleLookup) {
+         // handlers for each war and init params for the web api
+        final List<WebAppContext> webAppContexts = new ArrayList<>();
         final Map<String, String> mimeMappings = new HashMap<>();
+        final Collection<WebAppContext> componentUiExtensionWebContexts = new ArrayList<>();
+        final Collection<WebAppContext> contentViewerWebContexts = new ArrayList<>();
+        final Map<String, List<UiExtension>> componentUiExtensionsByType = new HashMap<>();
+
         final ClassLoader frameworkClassLoader = getClass().getClassLoader();
         final ClassLoader jettyClassLoader = frameworkClassLoader.getParent();
 
         // deploy the other wars
-        if (CollectionUtils.isNotEmpty(otherWars)) {
-            // hold onto to the web contexts for all ui extensions
-            componentUiExtensionWebContexts = new ArrayList<>();
-            contentViewerWebContexts = new ArrayList<>();
-
+        if (!warToBundleLookup.isEmpty()) {
             // ui extension organized by component type
-            final Map<String, List<UiExtension>> componentUiExtensionsByType = new HashMap<>();
-            for (File war : otherWars) {
+            for (Map.Entry<File,Bundle> warBundleEntry : warToBundleLookup.entrySet()) {
+                final File war = warBundleEntry.getKey();
+                final Bundle warBundle = warBundleEntry.getValue();
+
                 // identify all known extension types in the war
                 final Map<UiExtensionType, List<String>> uiExtensionInWar = new HashMap<>();
                 identifyUiExtensionsForComponents(uiExtensionInWar, war);
@@ -248,7 +370,7 @@ public class JettyServer implements NiFiServer {
                     String warContextPath = String.format("/%s", warName);
 
                     // get the classloader for this war
-                    ClassLoader narClassLoaderForWar = warToBundleLookup.get(war).getClassLoader();
+                    ClassLoader narClassLoaderForWar = warBundle.getClassLoader();
 
                     // this should never be null
                     if (narClassLoaderForWar == null) {
@@ -302,51 +424,12 @@ public class JettyServer implements NiFiServer {
                     }
 
                     // include custom ui web context in the handlers
-                    handlers.addHandler(extensionUiContext);
+                    webAppContexts.add(extensionUiContext);
                 }
-
             }
-
-            // record all ui extensions to give to the web api
-            componentUiExtensions = new UiExtensionMapping(componentUiExtensionsByType);
-        } else {
-            componentUiExtensions = new UiExtensionMapping(Collections.EMPTY_MAP);
         }
 
-        // load the web ui app
-        final WebAppContext webUiContext = loadWar(webUiWar, "/nifi", frameworkClassLoader);
-        webUiContext.getInitParams().put("oidc-supported", String.valueOf(props.isOidcEnabled()));
-        webUiContext.getInitParams().put("knox-supported", String.valueOf(props.isKnoxSsoEnabled()));
-        webUiContext.getInitParams().put("whitelistedContextPaths", props.getWhitelistedContextPaths());
-        handlers.addHandler(webUiContext);
-
-        // load the web api app
-        webApiContext = loadWar(webApiWar, "/nifi-api", frameworkClassLoader);
-        handlers.addHandler(webApiContext);
-
-        // load the content viewer app
-        webContentViewerContext = loadWar(webContentViewerWar, "/nifi-content-viewer", frameworkClassLoader);
-        webContentViewerContext.getInitParams().putAll(mimeMappings);
-        handlers.addHandler(webContentViewerContext);
-
-        // create a web app for the docs
-        final String docsContextPath = "/nifi-docs";
-
-        // load the documentation war
-        webDocsContext = loadWar(webDocsWar, docsContextPath, frameworkClassLoader);
-
-        // add the servlets which serve the HTML documentation within the documentation web app
-        addDocsServlets(webDocsContext);
-
-        handlers.addHandler(webDocsContext);
-
-        // load the web error app
-        final WebAppContext webErrorContext = loadWar(webErrorWar, "/", frameworkClassLoader);
-        webErrorContext.getInitParams().put("whitelistedContextPaths", props.getWhitelistedContextPaths());
-        handlers.addHandler(webErrorContext);
-
-        // deploy the web apps
-        return gzip(handlers);
+        return new ExtensionUiInfo(webAppContexts, mimeMappings, componentUiExtensionWebContexts, contentViewerWebContexts, componentUiExtensionsByType);
     }
 
     /**
@@ -472,7 +555,7 @@ public class JettyServer implements NiFiServer {
         webappContext.setDisplayName(contextPath);
 
         // instruction jetty to examine these jars for tlds, web-fragments, etc
-        webappContext.setAttribute("org.eclipse.jetty.server.webapp.ContainerIncludeJarPattern", ".*/[^/]*servlet-api-[^/]*\\.jar$|.*/javax.servlet.jsp.jstl-.*\\\\.jar$|.*/[^/]*taglibs.*\\.jar$");
+        webappContext.setAttribute(CONTAINER_INCLUDE_PATTERN_KEY, CONTAINER_INCLUDE_PATTERN_VALUE);
 
         // remove slf4j server class to allow WAR files to have slf4j dependencies in WEB-INF/lib
         List<String> serverClasses = new ArrayList<>(Arrays.asList(webappContext.getServerClasses()));
@@ -835,10 +918,16 @@ public class JettyServer implements NiFiServer {
     @Override
     public void start() {
         try {
-            ExtensionManager.discoverExtensions(systemBundle, bundles);
-            ExtensionManager.logClassLoaderMapping();
+            // Create a standard extension manager and discover extensions
+            final ExtensionDiscoveringManager extensionManager = new StandardExtensionDiscoveringManager();
+            extensionManager.discoverExtensions(systemBundle, bundles);
+            extensionManager.logClassLoaderMapping();
 
-            DocGenerator.generate(props, extensionMapping);
+            // Set the extension manager into the holder which makes it available to the Spring context via a factory bean
+            ExtensionManagerHolder.init(extensionManager);
+
+            // Generate docs for extensions
+            DocGenerator.generate(props, extensionManager, extensionMapping);
 
             // start the server
             server.start();
@@ -866,34 +955,14 @@ public class JettyServer implements NiFiServer {
 
                 // get the application context
                 final WebApplicationContext webApplicationContext = WebApplicationContextUtils.getRequiredWebApplicationContext(webApiServletContext);
+                final NiFiWebConfigurationContext configurationContext = webApplicationContext.getBean("nifiWebConfigurationContext", NiFiWebConfigurationContext.class);
+                final FilterHolder securityFilter = webApiContext.getServletHandler().getFilter("springSecurityFilterChain");
 
                 // component ui extensions
-                if (CollectionUtils.isNotEmpty(componentUiExtensionWebContexts)) {
-                    final NiFiWebConfigurationContext configurationContext = webApplicationContext.getBean("nifiWebConfigurationContext", NiFiWebConfigurationContext.class);
-
-                    for (final WebAppContext customUiContext : componentUiExtensionWebContexts) {
-                        // set the NiFi context in each custom ui servlet context
-                        final ServletContext customUiServletContext = customUiContext.getServletHandler().getServletContext();
-                        customUiServletContext.setAttribute("nifi-web-configuration-context", configurationContext);
-
-                        // add the security filter to any ui extensions wars
-                        final FilterHolder securityFilter = webApiContext.getServletHandler().getFilter("springSecurityFilterChain");
-                        if (securityFilter != null) {
-                            customUiContext.addFilter(securityFilter, "/*", EnumSet.allOf(DispatcherType.class));
-                        }
-                    }
-                }
+                performInjectionForComponentUis(componentUiExtensionWebContexts, configurationContext, securityFilter);
 
                 // content viewer extensions
-                if (CollectionUtils.isNotEmpty(contentViewerWebContexts)) {
-                    for (final WebAppContext contentViewerContext : contentViewerWebContexts) {
-                        // add the security filter to any content viewer  wars
-                        final FilterHolder securityFilter = webApiContext.getServletHandler().getFilter("springSecurityFilterChain");
-                        if (securityFilter != null) {
-                            contentViewerContext.addFilter(securityFilter, "/*", EnumSet.allOf(DispatcherType.class));
-                        }
-                    }
-                }
+                performInjectionForContentViewerUis(contentViewerWebContexts, securityFilter);
 
                 // content viewer controller
                 if (webContentViewerContext != null) {
@@ -903,7 +972,6 @@ public class JettyServer implements NiFiServer {
                     final ServletContext webContentViewerServletContext = webContentViewerContext.getServletHandler().getServletContext();
                     webContentViewerServletContext.setAttribute("nifi-content-access", contentAccess);
 
-                    final FilterHolder securityFilter = webApiContext.getServletHandler().getFilter("springSecurityFilterChain");
                     if (securityFilter != null) {
                         webContentViewerContext.addFilter(securityFilter, "/*", EnumSet.allOf(DispatcherType.class));
                     }
@@ -950,6 +1018,17 @@ public class JettyServer implements NiFiServer {
                 }
             }
 
+            final NarLoader narLoader = new StandardNarLoader(
+                    props.getExtensionsWorkingDirectory(),
+                    props.getComponentDocumentationWorkingDirectory(),
+                    NarClassLoadersHolder.getInstance(),
+                    extensionManager,
+                    extensionMapping,
+                    this);
+
+            narAutoLoader = new NarAutoLoader(props.getNarAutoLoadDirectory(), narLoader);
+            narAutoLoader.start();
+
             // dump the application url after confirming everything started successfully
             dumpUrls();
         } catch (Exception ex) {
@@ -957,6 +1036,34 @@ public class JettyServer implements NiFiServer {
         }
     }
 
+    private void performInjectionForComponentUis(final Collection<WebAppContext> componentUiExtensionWebContexts,
+                                                 final NiFiWebConfigurationContext configurationContext, final FilterHolder securityFilter) {
+        if (CollectionUtils.isNotEmpty(componentUiExtensionWebContexts)) {
+            for (final WebAppContext customUiContext : componentUiExtensionWebContexts) {
+                // set the NiFi context in each custom ui servlet context
+                final ServletContext customUiServletContext = customUiContext.getServletHandler().getServletContext();
+                customUiServletContext.setAttribute("nifi-web-configuration-context", configurationContext);
+
+                // add the security filter to any ui extensions wars
+                if (securityFilter != null) {
+                    customUiContext.addFilter(securityFilter, "/*", EnumSet.allOf(DispatcherType.class));
+                }
+            }
+        }
+    }
+
+    private void performInjectionForContentViewerUis(final Collection<WebAppContext> contentViewerWebContexts,
+                                                     final FilterHolder securityFilter) {
+        if (CollectionUtils.isNotEmpty(contentViewerWebContexts)) {
+            for (final WebAppContext contentViewerContext : contentViewerWebContexts) {
+                // add the security filter to any content viewer  wars
+                if (securityFilter != null) {
+                    contentViewerContext.addFilter(securityFilter, "/*", EnumSet.allOf(DispatcherType.class));
+                }
+            }
+        }
+    }
+
     private void dumpUrls() throws SocketException {
         final List<String> urls = new ArrayList<>();
 
@@ -1031,6 +1138,11 @@ public class JettyServer implements NiFiServer {
         } catch (Exception ex) {
             logger.warn("Failed to stop web server", ex);
         }
+        try {
+            narAutoLoader.stop();
+        } catch (Exception e) {
+            logger.warn("Failed to stop NAR auto-loader", e);
+        }
     }
 
     private static final Filter FRAME_OPTIONS_FILTER = new Filter() {
@@ -1056,6 +1168,50 @@ public class JettyServer implements NiFiServer {
         public void destroy() {
         }
     };
+
+    /**
+     * Holds the result of loading WARs for custom UIs.
+     */
+    private static class ExtensionUiInfo {
+
+        private final Collection<WebAppContext> webAppContexts;
+        private final Map<String, String> mimeMappings;
+        private final Collection<WebAppContext> componentUiExtensionWebContexts;
+        private final Collection<WebAppContext> contentViewerWebContexts;
+        private final Map<String, List<UiExtension>> componentUiExtensionsByType;
+
+        public ExtensionUiInfo(final Collection<WebAppContext> webAppContexts,
+                               final Map<String, String> mimeMappings,
+                               final Collection<WebAppContext> componentUiExtensionWebContexts,
+                               final Collection<WebAppContext> contentViewerWebContexts,
+                               final Map<String, List<UiExtension>> componentUiExtensionsByType) {
+            this.webAppContexts = webAppContexts;
+            this.mimeMappings = mimeMappings;
+            this.componentUiExtensionWebContexts = componentUiExtensionWebContexts;
+            this.contentViewerWebContexts = contentViewerWebContexts;
+            this.componentUiExtensionsByType = componentUiExtensionsByType;
+        }
+
+        public Collection<WebAppContext> getWebAppContexts() {
+            return webAppContexts;
+        }
+
+        public Map<String, String> getMimeMappings() {
+            return mimeMappings;
+        }
+
+        public Collection<WebAppContext> getComponentUiExtensionWebContexts() {
+            return componentUiExtensionWebContexts;
+        }
+
+        public Collection<WebAppContext> getContentViewerWebContexts() {
+            return contentViewerWebContexts;
+        }
+
+        public Map<String, List<UiExtension>> getComponentUiExtensionsByType() {
+            return componentUiExtensionsByType;
+        }
+    }
 }
 
 @FunctionalInterface

http://git-wip-us.apache.org/repos/asf/nifi/blob/fdd8cdbb/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-ui-extension/src/main/java/org/apache/nifi/ui/extension/UiExtensionMapping.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-ui-extension/src/main/java/org/apache/nifi/ui/extension/UiExtensionMapping.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-ui-extension/src/main/java/org/apache/nifi/ui/extension/UiExtensionMapping.java
index 4202830..7122611 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-ui-extension/src/main/java/org/apache/nifi/ui/extension/UiExtensionMapping.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-ui-extension/src/main/java/org/apache/nifi/ui/extension/UiExtensionMapping.java
@@ -18,16 +18,17 @@ package org.apache.nifi.ui.extension;
 
 import java.util.List;
 import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
 
 /**
  * Mapping of all discovered UI extensions.
  */
 public class UiExtensionMapping {
 
-    private final Map<String, List<UiExtension>> uiExtensions;
+    private final Map<String, List<UiExtension>> uiExtensions = new ConcurrentHashMap<>();
 
     public UiExtensionMapping(Map<String, List<UiExtension>> uiExtensions) {
-        this.uiExtensions = uiExtensions;
+        this.uiExtensions.putAll(uiExtensions);
     }
 
     private String getBundleSpecificKey(final String type, final String bundleGroup, final String bundleArtifact, final String bundleVersion) {
@@ -72,4 +73,13 @@ public class UiExtensionMapping {
         return uiExtensions.get(type);
     }
 
+    /**
+     * Adds additional UI extensions to the mapping.
+     *
+     * @param uiExtensions the additional UI extension mappings.
+     */
+    public void addUiExtensions(Map<String, List<UiExtension>> uiExtensions) {
+        this.uiExtensions.putAll(uiExtensions);
+    }
+
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/fdd8cdbb/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 2d79006..42da0cc 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
@@ -190,7 +190,7 @@ class StandardAuthorizableLookup implements AuthorizableLookup {
     @Override
     public ComponentAuthorizable getConfigurableComponent(ConfigurableComponent configurableComponent) {
         try {
-            return new ConfigurableComponentAuthorizable(configurableComponent);
+            return new ConfigurableComponentAuthorizable(configurableComponent, controllerFacade.getExtensionManager());
         } catch (final Exception e) {
             throw new AccessDeniedException("Unable to create component to verify if it references any Controller Services.");
         }
@@ -199,7 +199,7 @@ class StandardAuthorizableLookup implements AuthorizableLookup {
     @Override
     public ComponentAuthorizable getProcessor(final String id) {
         final ProcessorNode processorNode = processorDAO.getProcessor(id);
-        return new ProcessorComponentAuthorizable(processorNode);
+        return new ProcessorComponentAuthorizable(processorNode, controllerFacade.getExtensionManager());
     }
 
     @Override
@@ -277,7 +277,7 @@ class StandardAuthorizableLookup implements AuthorizableLookup {
     @Override
     public ProcessGroupAuthorizable getProcessGroup(final String id) {
         final ProcessGroup processGroup = processGroupDAO.getProcessGroup(id);
-        return new StandardProcessGroupAuthorizable(processGroup);
+        return new StandardProcessGroupAuthorizable(processGroup, controllerFacade.getExtensionManager());
     }
 
     @Override
@@ -298,7 +298,7 @@ class StandardAuthorizableLookup implements AuthorizableLookup {
     @Override
     public ComponentAuthorizable getControllerService(final String id) {
         final ControllerServiceNode controllerService = controllerServiceDAO.getControllerService(id);
-        return new ControllerServiceComponentAuthorizable(controllerService);
+        return new ControllerServiceComponentAuthorizable(controllerService, controllerFacade.getExtensionManager());
     }
 
     @Override
@@ -362,7 +362,7 @@ class StandardAuthorizableLookup implements AuthorizableLookup {
     @Override
     public ComponentAuthorizable getReportingTask(final String id) {
         final ReportingTaskNode reportingTaskNode = reportingTaskDAO.getReportingTask(id);
-        return new ReportingTaskComponentAuthorizable(reportingTaskNode);
+        return new ReportingTaskComponentAuthorizable(reportingTaskNode, controllerFacade.getExtensionManager());
     }
 
     @Override
@@ -667,7 +667,8 @@ class StandardAuthorizableLookup implements AuthorizableLookup {
      */
     private void createTemporaryProcessorsAndControllerServices(final FlowSnippetDTO snippet,
                                                                 final Set<ComponentAuthorizable> processors,
-                                                                final Set<ComponentAuthorizable> controllerServices) {
+                                                                final Set<ComponentAuthorizable> controllerServices,
+                                                                final ExtensionManager extensionManager) {
 
         if (snippet == null) {
             return;
@@ -676,7 +677,7 @@ class StandardAuthorizableLookup implements AuthorizableLookup {
         if (snippet.getProcessors() != null) {
             snippet.getProcessors().forEach(processor -> {
                 try {
-                    final BundleCoordinate bundle = BundleUtils.getCompatibleBundle(processor.getType(), processor.getBundle());
+                    final BundleCoordinate bundle = BundleUtils.getCompatibleBundle(extensionManager, processor.getType(), processor.getBundle());
                     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
@@ -687,7 +688,7 @@ class StandardAuthorizableLookup implements AuthorizableLookup {
         if (snippet.getControllerServices() != null) {
             snippet.getControllerServices().forEach(controllerService -> {
                 try {
-                    final BundleCoordinate bundle = BundleUtils.getCompatibleBundle(controllerService.getType(), controllerService.getBundle());
+                    final BundleCoordinate bundle = BundleUtils.getCompatibleBundle(extensionManager, controllerService.getType(), controllerService.getBundle());
                     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
@@ -696,7 +697,7 @@ class StandardAuthorizableLookup implements AuthorizableLookup {
         }
 
         if (snippet.getProcessGroups() != null) {
-            snippet.getProcessGroups().stream().forEach(group -> createTemporaryProcessorsAndControllerServices(group.getContents(), processors, controllerServices));
+            snippet.getProcessGroups().stream().forEach(group -> createTemporaryProcessorsAndControllerServices(group.getContents(), processors, controllerServices, extensionManager));
         }
     }
 
@@ -712,7 +713,8 @@ class StandardAuthorizableLookup implements AuthorizableLookup {
         final Set<ComponentAuthorizable> controllerServices = new HashSet<>();
 
         // find all processors and controller services
-        createTemporaryProcessorsAndControllerServices(snippet, processors, controllerServices);
+        final ExtensionManager extensionManager = controllerFacade.getExtensionManager();
+        createTemporaryProcessorsAndControllerServices(snippet, processors, controllerServices, extensionManager);
 
         return new TemplateContentsAuthorizable() {
             @Override
@@ -758,9 +760,11 @@ class StandardAuthorizableLookup implements AuthorizableLookup {
      */
     private static class ConfigurableComponentAuthorizable implements ComponentAuthorizable {
         private final ConfigurableComponent configurableComponent;
+        private final ExtensionManager extensionManager;
 
-        public ConfigurableComponentAuthorizable(final ConfigurableComponent configurableComponent) {
+        public ConfigurableComponentAuthorizable(final ConfigurableComponent configurableComponent, final ExtensionManager extensionManager) {
             this.configurableComponent = configurableComponent;
+            this.extensionManager = extensionManager;
         }
 
         @Override
@@ -795,7 +799,7 @@ class StandardAuthorizableLookup implements AuthorizableLookup {
 
         @Override
         public void cleanUpResources() {
-            ExtensionManager.removeInstanceClassLoader(configurableComponent.getIdentifier());
+            extensionManager.removeInstanceClassLoader(configurableComponent.getIdentifier());
         }
     }
 
@@ -804,9 +808,11 @@ class StandardAuthorizableLookup implements AuthorizableLookup {
      */
     private static class ProcessorComponentAuthorizable implements ComponentAuthorizable {
         private final ProcessorNode processorNode;
+        private final ExtensionManager extensionManager;
 
-        public ProcessorComponentAuthorizable(ProcessorNode processorNode) {
+        public ProcessorComponentAuthorizable(final ProcessorNode processorNode, final ExtensionManager extensionManager) {
             this.processorNode = processorNode;
+            this.extensionManager = extensionManager;
         }
 
         @Override
@@ -841,7 +847,7 @@ class StandardAuthorizableLookup implements AuthorizableLookup {
 
         @Override
         public void cleanUpResources() {
-            ExtensionManager.removeInstanceClassLoader(processorNode.getIdentifier());
+            extensionManager.removeInstanceClassLoader(processorNode.getIdentifier());
         }
     }
 
@@ -850,9 +856,11 @@ class StandardAuthorizableLookup implements AuthorizableLookup {
      */
     private static class ControllerServiceComponentAuthorizable implements ComponentAuthorizable {
         private final ControllerServiceNode controllerServiceNode;
+        private final ExtensionManager extensionManager;
 
-        public ControllerServiceComponentAuthorizable(ControllerServiceNode controllerServiceNode) {
+        public ControllerServiceComponentAuthorizable(final ControllerServiceNode controllerServiceNode, final ExtensionManager extensionManager) {
             this.controllerServiceNode = controllerServiceNode;
+            this.extensionManager = extensionManager;
         }
 
         @Override
@@ -887,7 +895,7 @@ class StandardAuthorizableLookup implements AuthorizableLookup {
 
         @Override
         public void cleanUpResources() {
-            ExtensionManager.removeInstanceClassLoader(controllerServiceNode.getIdentifier());
+            extensionManager.removeInstanceClassLoader(controllerServiceNode.getIdentifier());
         }
     }
 
@@ -896,9 +904,11 @@ class StandardAuthorizableLookup implements AuthorizableLookup {
      */
     private static class ReportingTaskComponentAuthorizable implements ComponentAuthorizable {
         private final ReportingTaskNode reportingTaskNode;
+        private final ExtensionManager extensionManager;
 
-        public ReportingTaskComponentAuthorizable(ReportingTaskNode reportingTaskNode) {
+        public ReportingTaskComponentAuthorizable(final ReportingTaskNode reportingTaskNode, final ExtensionManager extensionManager) {
             this.reportingTaskNode = reportingTaskNode;
+            this.extensionManager = extensionManager;
         }
 
         @Override
@@ -933,15 +943,17 @@ class StandardAuthorizableLookup implements AuthorizableLookup {
 
         @Override
         public void cleanUpResources() {
-            ExtensionManager.removeInstanceClassLoader(reportingTaskNode.getIdentifier());
+            extensionManager.removeInstanceClassLoader(reportingTaskNode.getIdentifier());
         }
     }
 
     private static class StandardProcessGroupAuthorizable implements ProcessGroupAuthorizable {
         private final ProcessGroup processGroup;
+        private final ExtensionManager extensionManager;
 
-        public StandardProcessGroupAuthorizable(ProcessGroup processGroup) {
+        public StandardProcessGroupAuthorizable(final ProcessGroup processGroup, final ExtensionManager extensionManager) {
             this.processGroup = processGroup;
+            this.extensionManager = extensionManager;
         }
 
         @Override
@@ -952,7 +964,7 @@ class StandardAuthorizableLookup implements AuthorizableLookup {
         @Override
         public Set<ComponentAuthorizable> getEncapsulatedProcessors() {
             return processGroup.findAllProcessors().stream().map(
-                    processorNode -> new ProcessorComponentAuthorizable(processorNode)).collect(Collectors.toSet());
+                    processorNode -> new ProcessorComponentAuthorizable(processorNode, extensionManager)).collect(Collectors.toSet());
         }
 
         @Override
@@ -984,7 +996,7 @@ class StandardAuthorizableLookup implements AuthorizableLookup {
         @Override
         public Set<ProcessGroupAuthorizable> getEncapsulatedProcessGroups() {
             return processGroup.findAllProcessGroups().stream().map(
-                    group -> new StandardProcessGroupAuthorizable(group)).collect(Collectors.toSet());
+                    group -> new StandardProcessGroupAuthorizable(group, extensionManager)).collect(Collectors.toSet());
         }
 
         @Override
@@ -1000,7 +1012,7 @@ class StandardAuthorizableLookup implements AuthorizableLookup {
         @Override
         public Set<ComponentAuthorizable> getEncapsulatedControllerServices() {
             return processGroup.findAllControllerServices().stream().map(
-                    controllerServiceNode -> new ControllerServiceComponentAuthorizable(controllerServiceNode)).collect(Collectors.toSet());
+                    controllerServiceNode -> new ControllerServiceComponentAuthorizable(controllerServiceNode, extensionManager)).collect(Collectors.toSet());
         }
     }
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/fdd8cdbb/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/registry/flow/FlowRegistryUtils.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/registry/flow/FlowRegistryUtils.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/registry/flow/FlowRegistryUtils.java
index 1125d9c..cd48ba7 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/registry/flow/FlowRegistryUtils.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/registry/flow/FlowRegistryUtils.java
@@ -20,9 +20,9 @@ package org.apache.nifi.registry.flow;
 import org.apache.nifi.annotation.behavior.Restricted;
 import org.apache.nifi.bundle.BundleCoordinate;
 import org.apache.nifi.components.ConfigurableComponent;
-import org.apache.nifi.nar.ExtensionManager;
 import org.apache.nifi.util.Tuple;
 import org.apache.nifi.web.NiFiCoreException;
+import org.apache.nifi.web.NiFiServiceFacade;
 import org.apache.nifi.web.api.dto.BundleDTO;
 
 import java.util.HashSet;
@@ -30,14 +30,14 @@ import java.util.Set;
 
 public class FlowRegistryUtils {
 
-    public static Set<ConfigurableComponent> getRestrictedComponents(final VersionedProcessGroup group) {
+    public static Set<ConfigurableComponent> getRestrictedComponents(final VersionedProcessGroup group, final NiFiServiceFacade serviceFacade) {
         final Set<ConfigurableComponent> restrictedComponents = new HashSet<>();
 
         final Set<Tuple<String, BundleCoordinate>> componentTypes = new HashSet<>();
         populateComponentTypes(group, componentTypes);
 
         for (final Tuple<String, BundleCoordinate> tuple : componentTypes) {
-            final ConfigurableComponent component = ExtensionManager.getTempComponent(tuple.getKey(), tuple.getValue());
+            final ConfigurableComponent component = serviceFacade.getTempComponent(tuple.getKey(), tuple.getValue());
             if (component == null) {
                 throw new NiFiCoreException("Could not create an instance of component " + tuple.getKey() + " using bundle coordinates " + tuple.getValue());
             }

http://git-wip-us.apache.org/repos/asf/nifi/blob/fdd8cdbb/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/NiFiServiceFacade.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/NiFiServiceFacade.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/NiFiServiceFacade.java
index 9028f41..2068c31 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/NiFiServiceFacade.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/NiFiServiceFacade.java
@@ -19,6 +19,8 @@ package org.apache.nifi.web;
 import org.apache.nifi.authorization.AuthorizeAccess;
 import org.apache.nifi.authorization.RequestAction;
 import org.apache.nifi.authorization.user.NiFiUser;
+import org.apache.nifi.bundle.BundleCoordinate;
+import org.apache.nifi.components.ConfigurableComponent;
 import org.apache.nifi.controller.ScheduledState;
 import org.apache.nifi.controller.repository.claim.ContentDirection;
 import org.apache.nifi.controller.service.ControllerServiceState;
@@ -31,6 +33,7 @@ import org.apache.nifi.web.api.dto.AffectedComponentDTO;
 import org.apache.nifi.web.api.dto.BulletinBoardDTO;
 import org.apache.nifi.web.api.dto.BulletinDTO;
 import org.apache.nifi.web.api.dto.BulletinQueryDTO;
+import org.apache.nifi.web.api.dto.BundleDTO;
 import org.apache.nifi.web.api.dto.ClusterDTO;
 import org.apache.nifi.web.api.dto.ComponentHistoryDTO;
 import org.apache.nifi.web.api.dto.ComponentStateDTO;
@@ -2117,4 +2120,32 @@ public interface NiFiServiceFacade {
      * @return the resources
      */
     List<ResourceDTO> getResources();
+
+    // ----------------------------------------
+    // Bundle methods
+    // ----------------------------------------
+
+    /**
+     * Discovers the compatible bundle details for the components in the specified Versioned Process Group and updates the Versioned Process Group
+     * to reflect the appropriate bundles.
+     *
+     * @param versionedGroup the versioned group
+     */
+    void discoverCompatibleBundles(VersionedProcessGroup versionedGroup);
+
+    /**
+     * @param type the component type
+     * @param bundleDTO bundle to find the component
+     * @return the bundle coordinate
+     * @throws IllegalStateException no compatible bundle found
+     */
+    BundleCoordinate getCompatibleBundle(String type, BundleDTO bundleDTO);
+
+    /**
+     * @param classType the class name
+     * @param bundleCoordinate the bundle coordinate
+     * @return the temp component
+     */
+    ConfigurableComponent getTempComponent(String classType, BundleCoordinate bundleCoordinate);
+
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/fdd8cdbb/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 7e7d132..a960dbf 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
@@ -43,6 +43,7 @@ import org.apache.nifi.authorization.resource.ResourceFactory;
 import org.apache.nifi.authorization.resource.OperationAuthorizable;
 import org.apache.nifi.authorization.user.NiFiUser;
 import org.apache.nifi.authorization.user.NiFiUserUtils;
+import org.apache.nifi.bundle.BundleCoordinate;
 import org.apache.nifi.cluster.coordination.ClusterCoordinator;
 import org.apache.nifi.cluster.coordination.heartbeat.HeartbeatMonitor;
 import org.apache.nifi.cluster.coordination.heartbeat.NodeHeartbeat;
@@ -126,6 +127,7 @@ import org.apache.nifi.reporting.Bulletin;
 import org.apache.nifi.reporting.BulletinQuery;
 import org.apache.nifi.reporting.BulletinRepository;
 import org.apache.nifi.reporting.ComponentType;
+import org.apache.nifi.util.BundleUtils;
 import org.apache.nifi.util.FlowDifferenceFilters;
 import org.apache.nifi.util.NiFiProperties;
 import org.apache.nifi.web.api.dto.AccessPolicyDTO;
@@ -135,6 +137,7 @@ import org.apache.nifi.web.api.dto.BucketDTO;
 import org.apache.nifi.web.api.dto.BulletinBoardDTO;
 import org.apache.nifi.web.api.dto.BulletinDTO;
 import org.apache.nifi.web.api.dto.BulletinQueryDTO;
+import org.apache.nifi.web.api.dto.BundleDTO;
 import org.apache.nifi.web.api.dto.ClusterDTO;
 import org.apache.nifi.web.api.dto.ComponentDTO;
 import org.apache.nifi.web.api.dto.ComponentDifferenceDTO;
@@ -3064,6 +3067,21 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
         return resourceDtos;
     }
 
+    @Override
+    public void discoverCompatibleBundles(VersionedProcessGroup versionedGroup) {
+        BundleUtils.discoverCompatibleBundles(controllerFacade.getExtensionManager(), versionedGroup);
+    }
+
+    @Override
+    public BundleCoordinate getCompatibleBundle(String type, BundleDTO bundleDTO) {
+        return BundleUtils.getCompatibleBundle(controllerFacade.getExtensionManager(), type, bundleDTO);
+    }
+
+    @Override
+    public ConfigurableComponent getTempComponent(String classType, BundleCoordinate bundleCoordinate) {
+        return controllerFacade.getExtensionManager().getTempComponent(classType, bundleCoordinate);
+    }
+
     /**
      * Ensures the specified user has permission to access the specified port. This method does
      * not utilize the DataTransferAuthorizable as that will enforce the entire chain is
@@ -3837,7 +3855,7 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
 
     private InstantiatedVersionedProcessGroup createFlowSnapshot(final String processGroupId) {
         final ProcessGroup processGroup = processGroupDAO.getProcessGroup(processGroupId);
-        final NiFiRegistryFlowMapper mapper = new NiFiRegistryFlowMapper();
+        final NiFiRegistryFlowMapper mapper = new NiFiRegistryFlowMapper(controllerFacade.getExtensionManager());
         final InstantiatedVersionedProcessGroup versionedGroup = mapper.mapProcessGroup(processGroup, controllerFacade.getControllerServiceProvider(), flowRegistryClient, false);
         return versionedGroup;
     }
@@ -3864,7 +3882,7 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
             throw new NiFiCoreException("Failed to retrieve flow with Flow Registry in order to calculate local differences due to " + e.getMessage(), e);
         }
 
-        final NiFiRegistryFlowMapper mapper = new NiFiRegistryFlowMapper();
+        final NiFiRegistryFlowMapper mapper = new NiFiRegistryFlowMapper(controllerFacade.getExtensionManager());
         final VersionedProcessGroup localGroup = mapper.mapProcessGroup(processGroup, controllerFacade.getControllerServiceProvider(), flowRegistryClient, true);
         final VersionedProcessGroup registryGroup = versionedFlowSnapshot.getFlowContents();
 
@@ -3998,7 +4016,7 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
     public Set<AffectedComponentEntity> getComponentsAffectedByVersionChange(final String processGroupId, final VersionedFlowSnapshot updatedSnapshot) {
         final ProcessGroup group = processGroupDAO.getProcessGroup(processGroupId);
 
-        final NiFiRegistryFlowMapper mapper = new NiFiRegistryFlowMapper();
+        final NiFiRegistryFlowMapper mapper = new NiFiRegistryFlowMapper(controllerFacade.getExtensionManager());
         final VersionedProcessGroup localContents = mapper.mapProcessGroup(group, controllerFacade.getControllerServiceProvider(), flowRegistryClient, true);
 
         final ComparableDataFlow localFlow = new StandardComparableDataFlow("Local Flow", localContents);

http://git-wip-us.apache.org/repos/asf/nifi/blob/fdd8cdbb/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/FlowResource.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/FlowResource.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/FlowResource.java
index aaa2027..4f94d98 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/FlowResource.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/FlowResource.java
@@ -42,7 +42,7 @@ import org.apache.nifi.controller.ScheduledState;
 import org.apache.nifi.controller.service.ControllerServiceNode;
 import org.apache.nifi.controller.service.ControllerServiceState;
 import org.apache.nifi.groups.ProcessGroup;
-import org.apache.nifi.nar.NarClassLoaders;
+import org.apache.nifi.nar.NarClassLoadersHolder;
 import org.apache.nifi.registry.client.NiFiRegistryException;
 import org.apache.nifi.util.NiFiProperties;
 import org.apache.nifi.web.IllegalClusterResourceRequestException;
@@ -1352,7 +1352,7 @@ public class FlowResource extends ApplicationResource {
         final NiFiProperties properties = getProperties();
         aboutDTO.setContentViewerUrl(properties.getProperty(NiFiProperties.CONTENT_VIEWER_URL));
 
-        final Bundle frameworkBundle = NarClassLoaders.getInstance().getFrameworkBundle();
+        final Bundle frameworkBundle = NarClassLoadersHolder.getInstance().getFrameworkBundle();
         if (frameworkBundle != null) {
             final BundleDetails frameworkDetails = frameworkBundle.getBundleDetails();
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/fdd8cdbb/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 91935c0..4aa5961 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
@@ -55,7 +55,6 @@ import org.apache.nifi.registry.variable.VariableRegistryUpdateRequest;
 import org.apache.nifi.registry.variable.VariableRegistryUpdateStep;
 import org.apache.nifi.remote.util.SiteToSiteRestApiClient;
 import org.apache.nifi.security.xml.XmlUtils;
-import org.apache.nifi.util.BundleUtils;
 import org.apache.nifi.web.NiFiServiceFacade;
 import org.apache.nifi.web.ResourceNotFoundException;
 import org.apache.nifi.web.Revision;
@@ -1700,7 +1699,7 @@ public class ProcessGroupResource extends ApplicationResource {
             versionControlInfo.setState(flowState.name());
 
             // Step 3: Resolve Bundle info
-            BundleUtils.discoverCompatibleBundles(flowSnapshot.getFlowContents());
+            serviceFacade.discoverCompatibleBundles(flowSnapshot.getFlowContents());
 
             // Step 4: Update contents of the ProcessGroupDTO passed in to include the components that need to be added.
             requestProcessGroupEntity.setVersionedFlowSnapshot(flowSnapshot);
@@ -1729,7 +1728,8 @@ public class ProcessGroupResource extends ApplicationResource {
                     // for write access to the RestrictedComponents resource
                     final VersionedFlowSnapshot versionedFlowSnapshot = requestProcessGroupEntity.getVersionedFlowSnapshot();
                     if (versionedFlowSnapshot != null) {
-                        final Set<ConfigurableComponent> restrictedComponents = FlowRegistryUtils.getRestrictedComponents(versionedFlowSnapshot.getFlowContents());
+                        final Set<ConfigurableComponent> restrictedComponents = FlowRegistryUtils.getRestrictedComponents(
+                                versionedFlowSnapshot.getFlowContents(), serviceFacade);
                         restrictedComponents.forEach(restrictedComponent -> {
                             final ComponentAuthorizable restrictedComponentAuthorizable = lookup.getConfigurableComponent(restrictedComponent);
                             authorizeRestrictions(authorizer, restrictedComponentAuthorizable);
@@ -3142,14 +3142,14 @@ public class ProcessGroupResource extends ApplicationResource {
     private void discoverCompatibleBundles(final FlowSnippetDTO snippet) {
         if (snippet.getProcessors() != null) {
             snippet.getProcessors().forEach(processor -> {
-                final BundleCoordinate coordinate = BundleUtils.getCompatibleBundle(processor.getType(), processor.getBundle());
+                final BundleCoordinate coordinate = serviceFacade.getCompatibleBundle(processor.getType(), processor.getBundle());
                 processor.setBundle(new BundleDTO(coordinate.getGroup(), coordinate.getId(), coordinate.getVersion()));
             });
         }
 
         if (snippet.getControllerServices() != null) {
             snippet.getControllerServices().forEach(controllerService -> {
-                final BundleCoordinate coordinate = BundleUtils.getCompatibleBundle(controllerService.getType(), controllerService.getBundle());
+                final BundleCoordinate coordinate = serviceFacade.getCompatibleBundle(controllerService.getType(), controllerService.getBundle());
                 controllerService.setBundle(new BundleDTO(coordinate.getGroup(), coordinate.getId(), coordinate.getVersion()));
             });
         }


[5/7] nifi git commit: NIFI-5673 Set up property/assembly for new auto-load directory - Set up NarAutoLoader to watch directory for new files - Move NarAutoLoader to JettyServer since it will need access to ExtensionManager - Created NarLoader to shared

Posted by ma...@apache.org.
http://git-wip-us.apache.org/repos/asf/nifi/blob/fdd8cdbb/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 1762068..06d32e2 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
@@ -118,15 +118,16 @@ public class StandardControllerServiceProvider implements ControllerServiceProvi
 
         ClassLoader cl = null;
         final ClassLoader currentContextClassLoader = Thread.currentThread().getContextClassLoader();
+        final ExtensionManager extensionManager = flowController.getExtensionManager();
         try {
             final Class<?> rawClass;
             try {
-                final Bundle csBundle = ExtensionManager.getBundle(bundleCoordinate);
+                final Bundle csBundle = extensionManager.getBundle(bundleCoordinate);
                 if (csBundle == null) {
                     throw new ControllerServiceInstantiationException("Unable to find bundle for coordinate " + bundleCoordinate.getCoordinate());
                 }
 
-                cl = ExtensionManager.createInstanceClassLoader(type, id, csBundle, additionalUrls);
+                cl = extensionManager.createInstanceClassLoader(type, id, csBundle, additionalUrls);
                 Thread.currentThread().setContextClassLoader(cl);
                 rawClass = Class.forName(type, false, cl);
             } catch (final Exception e) {
@@ -138,7 +139,7 @@ public class StandardControllerServiceProvider implements ControllerServiceProvi
             final Class<? extends ControllerService> controllerServiceClass = rawClass.asSubclass(ControllerService.class);
 
             final ControllerService originalService = controllerServiceClass.newInstance();
-            final StandardControllerServiceInvocationHandler invocationHandler = new StandardControllerServiceInvocationHandler(originalService);
+            final StandardControllerServiceInvocationHandler invocationHandler = new StandardControllerServiceInvocationHandler(extensionManager, originalService);
 
             // extract all interfaces... controllerServiceClass is non null so getAllInterfaces is non null
             final List<Class<?>> interfaceList = ClassUtils.getAllInterfaces(controllerServiceClass);
@@ -165,13 +166,13 @@ public class StandardControllerServiceProvider implements ControllerServiceProvi
             final ComponentVariableRegistry componentVarRegistry = new StandardComponentVariableRegistry(this.variableRegistry);
             final ValidationContextFactory validationContextFactory = new StandardValidationContextFactory(this, componentVarRegistry);
             final ControllerServiceNode serviceNode = new StandardControllerServiceNode(originalLoggableComponent, proxiedLoggableComponent, invocationHandler,
-                id, validationContextFactory, this, componentVarRegistry, flowController, validationTrigger);
+                id, validationContextFactory, this, componentVarRegistry, flowController, flowController.getExtensionManager(), validationTrigger);
             serviceNode.setName(rawClass.getSimpleName());
 
             invocationHandler.setServiceNode(serviceNode);
 
             if (firstTimeAdded) {
-                try (final NarCloseable x = NarCloseable.withComponentNarLoader(originalService.getClass(), originalService.getIdentifier())) {
+                try (final NarCloseable x = NarCloseable.withComponentNarLoader(flowController.getExtensionManager(), originalService.getClass(), originalService.getIdentifier())) {
                     ReflectionUtils.invokeMethodsWithAnnotation(OnAdded.class, originalService);
                 } catch (final Exception e) {
                     throw new ComponentLifeCycleException("Failed to invoke On-Added Lifecycle methods of " + originalService, e);
@@ -244,7 +245,8 @@ public class StandardControllerServiceProvider implements ControllerServiceProvi
 
         final ComponentVariableRegistry componentVarRegistry = new StandardComponentVariableRegistry(this.variableRegistry);
         final ControllerServiceNode serviceNode = new StandardControllerServiceNode(proxiedLoggableComponent, proxiedLoggableComponent, invocationHandler, id,
-            new StandardValidationContextFactory(this, variableRegistry), this, componentType, type, componentVarRegistry, flowController, validationTrigger, true);
+            new StandardValidationContextFactory(this, variableRegistry), this, componentType, type, componentVarRegistry, flowController,
+                flowController.getExtensionManager(), validationTrigger, true);
 
         serviceCache.putIfAbsent(id, serviceNode);
         return serviceNode;
@@ -710,7 +712,8 @@ public class StandardControllerServiceProvider implements ControllerServiceProvi
 
         group.removeControllerService(serviceNode);
         LogRepositoryFactory.removeRepository(serviceNode.getIdentifier());
-        ExtensionManager.removeInstanceClassLoader(serviceNode.getIdentifier());
+        final ExtensionManager extensionManager = flowController.getExtensionManager();
+        extensionManager.removeInstanceClassLoader(serviceNode.getIdentifier());
         serviceCache.remove(serviceNode.getIdentifier());
     }
 
@@ -819,4 +822,9 @@ public class StandardControllerServiceProvider implements ControllerServiceProvi
     public Set<String> getControllerServiceIdentifiers(final Class<? extends ControllerService> serviceType) throws IllegalArgumentException {
         throw new UnsupportedOperationException("Cannot obtain Controller Service Identifiers for service type " + serviceType + " without providing a Process Group Identifier");
     }
+
+    @Override
+    public ExtensionManager getExtensionManager() {
+        return flowController.getExtensionManager();
+    }
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/fdd8cdbb/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/state/manager/StandardStateManagerProvider.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/state/manager/StandardStateManagerProvider.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/state/manager/StandardStateManagerProvider.java
index 8464599..dbdbb6b 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/state/manager/StandardStateManagerProvider.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/state/manager/StandardStateManagerProvider.java
@@ -70,16 +70,17 @@ public class StandardStateManagerProvider implements StateManagerProvider{
         this.clusterStateProvider = clusterStateProvider;
     }
 
-    public static synchronized StateManagerProvider create(final NiFiProperties properties, final VariableRegistry variableRegistry) throws ConfigParseException, IOException {
+    public static synchronized StateManagerProvider create(final NiFiProperties properties, final VariableRegistry variableRegistry, final ExtensionManager extensionManager)
+            throws ConfigParseException, IOException {
         if (provider != null) {
             return provider;
         }
 
-        final StateProvider localProvider = createLocalStateProvider(properties,variableRegistry);
+        final StateProvider localProvider = createLocalStateProvider(properties,variableRegistry, extensionManager);
 
         final StateProvider clusterProvider;
         if (properties.isNode()) {
-            clusterProvider = createClusteredStateProvider(properties,variableRegistry);
+            clusterProvider = createClusteredStateProvider(properties,variableRegistry, extensionManager);
         } else {
             clusterProvider = null;
         }
@@ -88,20 +89,23 @@ public class StandardStateManagerProvider implements StateManagerProvider{
         return provider;
     }
 
-    private static StateProvider createLocalStateProvider(final NiFiProperties properties, final VariableRegistry variableRegistry) throws IOException, ConfigParseException {
+    private static StateProvider createLocalStateProvider(final NiFiProperties properties, final VariableRegistry variableRegistry, final ExtensionManager extensionManager)
+            throws IOException, ConfigParseException {
         final File configFile = properties.getStateManagementConfigFile();
-        return createStateProvider(configFile, Scope.LOCAL, properties, variableRegistry);
+        return createStateProvider(configFile, Scope.LOCAL, properties, variableRegistry, extensionManager);
     }
 
 
-    private static StateProvider createClusteredStateProvider(final NiFiProperties properties, final VariableRegistry variableRegistry) throws IOException, ConfigParseException {
+    private static StateProvider createClusteredStateProvider(final NiFiProperties properties, final VariableRegistry variableRegistry, final ExtensionManager extensionManager)
+            throws IOException, ConfigParseException {
         final File configFile = properties.getStateManagementConfigFile();
-        return createStateProvider(configFile, Scope.CLUSTER, properties, variableRegistry);
+        return createStateProvider(configFile, Scope.CLUSTER, properties, variableRegistry, extensionManager);
     }
 
 
     private static StateProvider createStateProvider(final File configFile, final Scope scope, final NiFiProperties properties,
-                                                     final VariableRegistry variableRegistry) throws ConfigParseException, IOException {
+                                                     final VariableRegistry variableRegistry, final ExtensionManager extensionManager)
+            throws ConfigParseException, IOException {
         final String providerId;
         final String providerIdPropertyName;
         final String providerDescription;
@@ -169,7 +173,7 @@ public class StandardStateManagerProvider implements StateManagerProvider{
 
         final StateProvider provider;
         try {
-            provider = instantiateStateProvider(providerClassName);
+            provider = instantiateStateProvider(extensionManager, providerClassName);
         } catch (final Exception e) {
             throw new RuntimeException("Cannot create " + providerDescription + " of type " + providerClassName, e);
         }
@@ -223,10 +227,10 @@ public class StandardStateManagerProvider implements StateManagerProvider{
         return provider;
     }
 
-    private static StateProvider instantiateStateProvider(final String type) throws ClassNotFoundException, InstantiationException, IllegalAccessException {
+    private static StateProvider instantiateStateProvider(final ExtensionManager extensionManager, final String type) throws ClassNotFoundException, InstantiationException, IllegalAccessException {
         final ClassLoader ctxClassLoader = Thread.currentThread().getContextClassLoader();
         try {
-            final List<Bundle> bundles = ExtensionManager.getBundles(type);
+            final List<Bundle> bundles = extensionManager.getBundles(type);
             if (bundles.size() == 0) {
                 throw new IllegalStateException(String.format("The specified class '%s' is not known to this nifi.", type));
             }

http://git-wip-us.apache.org/repos/asf/nifi/blob/fdd8cdbb/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/tasks/ConnectableTask.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/tasks/ConnectableTask.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/tasks/ConnectableTask.java
index e68aba8..851aad3 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/tasks/ConnectableTask.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/tasks/ConnectableTask.java
@@ -197,7 +197,7 @@ public class ConnectableTask {
 
         final String originalThreadName = Thread.currentThread().getName();
         try {
-            try (final AutoCloseable ncl = NarCloseable.withComponentNarLoader(connectable.getRunnableComponent().getClass(), connectable.getIdentifier())) {
+            try (final AutoCloseable ncl = NarCloseable.withComponentNarLoader(flowController.getExtensionManager(), connectable.getRunnableComponent().getClass(), connectable.getIdentifier())) {
                 boolean shouldRun = connectable.getScheduledState() == ScheduledState.RUNNING;
                 while (shouldRun) {
                     connectable.onTrigger(processContext, activeSessionFactory);

http://git-wip-us.apache.org/repos/asf/nifi/blob/fdd8cdbb/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/tasks/ReportingTaskWrapper.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/tasks/ReportingTaskWrapper.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/tasks/ReportingTaskWrapper.java
index b05321f..b60302a 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/tasks/ReportingTaskWrapper.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/tasks/ReportingTaskWrapper.java
@@ -20,6 +20,7 @@ import org.apache.nifi.annotation.lifecycle.OnStopped;
 import org.apache.nifi.controller.ReportingTaskNode;
 import org.apache.nifi.controller.scheduling.LifecycleState;
 import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.nar.ExtensionManager;
 import org.apache.nifi.nar.NarCloseable;
 import org.apache.nifi.processor.SimpleProcessLogger;
 import org.apache.nifi.util.ReflectionUtils;
@@ -28,16 +29,18 @@ public class ReportingTaskWrapper implements Runnable {
 
     private final ReportingTaskNode taskNode;
     private final LifecycleState lifecycleState;
+    private final ExtensionManager extensionManager;
 
-    public ReportingTaskWrapper(final ReportingTaskNode taskNode, final LifecycleState lifecycleState) {
+    public ReportingTaskWrapper(final ReportingTaskNode taskNode, final LifecycleState lifecycleState, final ExtensionManager extensionManager) {
         this.taskNode = taskNode;
         this.lifecycleState = lifecycleState;
+        this.extensionManager = extensionManager;
     }
 
     @Override
     public synchronized void run() {
         lifecycleState.incrementActiveThreadCount(null);
-        try (final NarCloseable narCloseable = NarCloseable.withComponentNarLoader(taskNode.getReportingTask().getClass(), taskNode.getIdentifier())) {
+        try (final NarCloseable narCloseable = NarCloseable.withComponentNarLoader(extensionManager, taskNode.getReportingTask().getClass(), taskNode.getIdentifier())) {
             taskNode.getReportingTask().onTrigger(taskNode.getReportingContext());
         } catch (final Throwable t) {
             final ComponentLog componentLog = new SimpleProcessLogger(taskNode.getIdentifier(), taskNode.getReportingTask());
@@ -50,7 +53,7 @@ public class ReportingTaskWrapper implements Runnable {
                 // if the reporting task is no longer scheduled to run and this is the last thread,
                 // invoke the OnStopped methods
                 if (!lifecycleState.isScheduled() && lifecycleState.getActiveThreadCount() == 1 && lifecycleState.mustCallOnStoppedMethods()) {
-                    try (final NarCloseable x = NarCloseable.withComponentNarLoader(taskNode.getReportingTask().getClass(), taskNode.getIdentifier())) {
+                    try (final NarCloseable x = NarCloseable.withComponentNarLoader(extensionManager, taskNode.getReportingTask().getClass(), taskNode.getIdentifier())) {
                         ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnStopped.class, taskNode.getReportingTask(), taskNode.getConfigurationContext());
                     }
                 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/fdd8cdbb/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 7da5702..999a42d 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
@@ -84,11 +84,13 @@ public class FingerprintFactory {
     private static final String ENCRYPTED_VALUE_SUFFIX = "}";
     private final StringEncryptor encryptor;
     private final DocumentBuilder flowConfigDocBuilder;
+    private final ExtensionManager extensionManager;
 
     private static final Logger logger = LoggerFactory.getLogger(FingerprintFactory.class);
 
-    public FingerprintFactory(final StringEncryptor encryptor) {
+    public FingerprintFactory(final StringEncryptor encryptor, final ExtensionManager extensionManager) {
         this.encryptor = encryptor;
+        this.extensionManager = extensionManager;
 
         final DocumentBuilderFactory documentBuilderFactory = DocumentBuilderFactory.newInstance();
         documentBuilderFactory.setNamespaceAware(true);
@@ -108,9 +110,10 @@ public class FingerprintFactory {
         }
     }
 
-    public FingerprintFactory(final StringEncryptor encryptor, final DocumentBuilder docBuilder) {
+    public FingerprintFactory(final StringEncryptor encryptor, final DocumentBuilder docBuilder, final ExtensionManager extensionManager) {
         this.encryptor = encryptor;
         this.flowConfigDocBuilder = docBuilder;
+        this.extensionManager = extensionManager;
     }
 
     /**
@@ -408,7 +411,7 @@ public class FingerprintFactory {
 
         // 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);
+        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);
         }
@@ -640,7 +643,7 @@ public class FingerprintFactory {
 
         // 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);
+        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());
         }
@@ -672,7 +675,7 @@ public class FingerprintFactory {
     private BundleCoordinate getCoordinate(final String type, final BundleDTO dto) {
         BundleCoordinate coordinate;
         try {
-            coordinate = BundleUtils.getCompatibleBundle(type, dto);
+            coordinate = BundleUtils.getCompatibleBundle(extensionManager, type, dto);
         } catch (final IllegalStateException e) {
             if (dto == null) {
                 coordinate = BundleCoordinate.UNKNOWN_COORDINATE;
@@ -697,7 +700,7 @@ public class FingerprintFactory {
 
         // 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);
+        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());
         }

http://git-wip-us.apache.org/repos/asf/nifi/blob/fdd8cdbb/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 a683a9e..a27962a 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
@@ -65,7 +65,6 @@ import org.apache.nifi.encrypt.StringEncryptor;
 import org.apache.nifi.flowfile.FlowFilePrioritizer;
 import org.apache.nifi.logging.LogLevel;
 import org.apache.nifi.logging.LogRepositoryFactory;
-import org.apache.nifi.nar.ExtensionManager;
 import org.apache.nifi.nar.NarCloseable;
 import org.apache.nifi.processor.Relationship;
 import org.apache.nifi.processor.StandardProcessContext;
@@ -465,7 +464,7 @@ public final class StandardProcessGroup implements ProcessGroup {
 
     private void shutdown(final ProcessGroup procGroup) {
         for (final ProcessorNode node : procGroup.getProcessors()) {
-            try (final NarCloseable x = NarCloseable.withComponentNarLoader(node.getProcessor().getClass(), node.getIdentifier())) {
+            try (final NarCloseable x = NarCloseable.withComponentNarLoader(flowController.getExtensionManager(), node.getProcessor().getClass(), node.getIdentifier())) {
                 final StandardProcessContext processContext = new StandardProcessContext(node, controllerServiceProvider, encryptor, getStateManager(node.getIdentifier()), () -> false);
                 ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnShutdown.class, node.getProcessor(), processContext);
             }
@@ -903,7 +902,7 @@ public final class StandardProcessGroup implements ProcessGroup {
                 conn.verifyCanDelete();
             }
 
-            try (final NarCloseable x = NarCloseable.withComponentNarLoader(processor.getProcessor().getClass(), processor.getIdentifier())) {
+            try (final NarCloseable x = NarCloseable.withComponentNarLoader(flowController.getExtensionManager(), processor.getProcessor().getClass(), processor.getIdentifier())) {
                 final StandardProcessContext processContext = new StandardProcessContext(processor, controllerServiceProvider, encryptor, getStateManager(processor.getIdentifier()), () -> false);
                 ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnRemoved.class, processor.getProcessor(), processContext);
             } catch (final Exception e) {
@@ -951,7 +950,7 @@ public final class StandardProcessGroup implements ProcessGroup {
             if (removed) {
                 try {
                     LogRepositoryFactory.removeRepository(processor.getIdentifier());
-                    ExtensionManager.removeInstanceClassLoader(id);
+                    flowController.getExtensionManager().removeInstanceClassLoader(id);
                 } catch (Throwable t) {
                 }
             }
@@ -2108,7 +2107,7 @@ public final class StandardProcessGroup implements ProcessGroup {
 
             service.verifyCanDelete();
 
-            try (final NarCloseable x = NarCloseable.withComponentNarLoader(service.getControllerServiceImplementation().getClass(), service.getIdentifier())) {
+            try (final NarCloseable x = NarCloseable.withComponentNarLoader(flowController.getExtensionManager(), service.getControllerServiceImplementation().getClass(), service.getIdentifier())) {
                 final ConfigurationContext configurationContext = new StandardConfigurationContext(service, controllerServiceProvider, null, variableRegistry);
                 ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnRemoved.class, service.getControllerServiceImplementation(), configurationContext);
             }
@@ -2149,7 +2148,7 @@ public final class StandardProcessGroup implements ProcessGroup {
         } finally {
             if (removed) {
                 try {
-                    ExtensionManager.removeInstanceClassLoader(service.getIdentifier());
+                    flowController.getExtensionManager().removeInstanceClassLoader(service.getIdentifier());
                 } catch (Throwable t) {
                 }
             }
@@ -3308,7 +3307,7 @@ public final class StandardProcessGroup implements ProcessGroup {
         try {
             verifyCanUpdate(proposedSnapshot, true, verifyNotDirty);
 
-            final NiFiRegistryFlowMapper mapper = new NiFiRegistryFlowMapper();
+            final NiFiRegistryFlowMapper mapper = new NiFiRegistryFlowMapper(flowController.getExtensionManager());
             final VersionedProcessGroup versionedGroup = mapper.mapProcessGroup(this, controllerServiceProvider, flowController.getFlowRegistryClient(), true);
 
             final ComparableDataFlow localFlow = new StandardComparableDataFlow("Local Flow", versionedGroup);
@@ -4348,7 +4347,7 @@ public final class StandardProcessGroup implements ProcessGroup {
             return null;
         }
 
-        final NiFiRegistryFlowMapper mapper = new NiFiRegistryFlowMapper();
+        final NiFiRegistryFlowMapper mapper = new NiFiRegistryFlowMapper(flowController.getExtensionManager());
         final VersionedProcessGroup versionedGroup = mapper.mapProcessGroup(this, controllerServiceProvider, flowController.getFlowRegistryClient(), false);
 
         final ComparableDataFlow currentFlow = new StandardComparableDataFlow("Local Flow", versionedGroup);
@@ -4505,7 +4504,7 @@ public final class StandardProcessGroup implements ProcessGroup {
                 final String processorToAddClass = processorToAdd.getType();
                 final BundleCoordinate processorToAddCoordinate = toCoordinate(processorToAdd.getBundle());
 
-                final boolean bundleExists = ExtensionManager.getBundles(processorToAddClass).stream()
+                final boolean bundleExists = flowController.getExtensionManager().getBundles(processorToAddClass).stream()
                         .anyMatch(b -> processorToAddCoordinate.equals(b.getBundleDetails().getCoordinate()));
 
                 if (!bundleExists) {
@@ -4525,7 +4524,7 @@ public final class StandardProcessGroup implements ProcessGroup {
                 final String serviceToAddClass = serviceToAdd.getType();
                 final BundleCoordinate serviceToAddCoordinate = toCoordinate(serviceToAdd.getBundle());
 
-                final boolean bundleExists = ExtensionManager.getBundles(serviceToAddClass).stream()
+                final boolean bundleExists = flowController.getExtensionManager().getBundles(serviceToAddClass).stream()
                         .anyMatch(b -> serviceToAddCoordinate.equals(b.getBundleDetails().getCoordinate()));
 
                 if (!bundleExists) {

http://git-wip-us.apache.org/repos/asf/nifi/blob/fdd8cdbb/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/persistence/StandardXMLFlowConfigurationDAO.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/persistence/StandardXMLFlowConfigurationDAO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/persistence/StandardXMLFlowConfigurationDAO.java
index 26c5224..e7e3ce8 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/persistence/StandardXMLFlowConfigurationDAO.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/persistence/StandardXMLFlowConfigurationDAO.java
@@ -36,6 +36,7 @@ import org.apache.nifi.controller.serialization.FlowSynchronizationException;
 import org.apache.nifi.controller.serialization.FlowSynchronizer;
 import org.apache.nifi.controller.serialization.StandardFlowSerializer;
 import org.apache.nifi.encrypt.StringEncryptor;
+import org.apache.nifi.nar.ExtensionManager;
 import org.apache.nifi.util.NiFiProperties;
 import org.apache.nifi.util.file.FileUtils;
 import org.slf4j.Logger;
@@ -47,10 +48,12 @@ public final class StandardXMLFlowConfigurationDAO implements FlowConfigurationD
     private final StringEncryptor encryptor;
     private final FlowConfigurationArchiveManager archiveManager;
     private final NiFiProperties nifiProperties;
+    private final ExtensionManager extensionManager;
 
     private static final Logger LOG = LoggerFactory.getLogger(StandardXMLFlowConfigurationDAO.class);
 
-    public StandardXMLFlowConfigurationDAO(final Path flowXml, final StringEncryptor encryptor, final NiFiProperties nifiProperties) throws IOException {
+    public StandardXMLFlowConfigurationDAO(final Path flowXml, final StringEncryptor encryptor, final NiFiProperties nifiProperties,
+                                           final ExtensionManager extensionManager) throws IOException {
         this.nifiProperties = nifiProperties;
         final File flowXmlFile = flowXml.toFile();
         if (!flowXmlFile.exists()) {
@@ -66,6 +69,7 @@ public final class StandardXMLFlowConfigurationDAO implements FlowConfigurationD
 
         this.flowXmlPath = flowXml;
         this.encryptor = encryptor;
+        this.extensionManager = extensionManager;
 
         this.archiveManager = new FlowConfigurationArchiveManager(flowXmlPath, nifiProperties);
     }
@@ -80,7 +84,7 @@ public final class StandardXMLFlowConfigurationDAO implements FlowConfigurationD
     public synchronized void load(final FlowController controller, final DataFlow dataFlow)
             throws IOException, FlowSerializationException, FlowSynchronizationException, UninheritableFlowException, MissingBundleException {
 
-        final FlowSynchronizer flowSynchronizer = new StandardFlowSynchronizer(encryptor, nifiProperties);
+        final FlowSynchronizer flowSynchronizer = new StandardFlowSynchronizer(encryptor, nifiProperties, extensionManager);
         controller.synchronize(flowSynchronizer, dataFlow);
 
         if (StandardFlowSynchronizer.isEmpty(dataFlow)) {

http://git-wip-us.apache.org/repos/asf/nifi/blob/fdd8cdbb/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/registry/flow/mapping/NiFiRegistryFlowMapper.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/registry/flow/mapping/NiFiRegistryFlowMapper.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/registry/flow/mapping/NiFiRegistryFlowMapper.java
index 074302a..c45d960 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/registry/flow/mapping/NiFiRegistryFlowMapper.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/registry/flow/mapping/NiFiRegistryFlowMapper.java
@@ -76,14 +76,21 @@ import java.util.stream.Collectors;
 
 
 public class NiFiRegistryFlowMapper {
+
+    private final ExtensionManager extensionManager;
+
     // We need to keep a mapping of component id to versionedComponentId as we transform these objects. This way, when
     // we call #mapConnectable, instead of generating a new UUID for the ConnectableComponent, we can lookup the 'versioned'
     // identifier based on the comopnent's actual id. We do connections last, so that all components will already have been
     // created before attempting to create the connection, where the ConnectableDTO is converted.
     private Map<String, String> versionedComponentIds = new HashMap<>();
 
+    public NiFiRegistryFlowMapper(final ExtensionManager extensionManager) {
+        this.extensionManager = extensionManager;
+    }
+
     public InstantiatedVersionedProcessGroup mapProcessGroup(final ProcessGroup group, final ControllerServiceProvider serviceProvider, final FlowRegistryClient registryClient,
-            final boolean mapDescendantVersionedFlows) {
+                                                             final boolean mapDescendantVersionedFlows) {
         versionedComponentIds.clear();
         final InstantiatedVersionedProcessGroup mapped = mapGroup(group, serviceProvider, registryClient, true, mapDescendantVersionedFlows);
 
@@ -385,7 +392,7 @@ public class NiFiRegistryFlowMapper {
 
         final List<ControllerServiceAPI> serviceApis = new ArrayList<>();
         for (final Class<?> serviceApiClass : serviceApiClasses) {
-            final BundleCoordinate bundleCoordinate = ExtensionManager.getBundle(serviceApiClass.getClassLoader()).getBundleDetails().getCoordinate();
+            final BundleCoordinate bundleCoordinate = extensionManager.getBundle(serviceApiClass.getClassLoader()).getBundleDetails().getCoordinate();
 
             final ControllerServiceAPI serviceApi = new ControllerServiceAPI();
             serviceApi.setType(serviceApiClass.getName());

http://git-wip-us.apache.org/repos/asf/nifi/blob/fdd8cdbb/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/spring/ExtensionManagerFactoryBean.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/spring/ExtensionManagerFactoryBean.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/spring/ExtensionManagerFactoryBean.java
new file mode 100644
index 0000000..5d8ba0f
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/spring/ExtensionManagerFactoryBean.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.spring;
+
+import org.apache.nifi.nar.ExtensionManager;
+import org.apache.nifi.nar.ExtensionManagerHolder;
+import org.apache.nifi.nar.StandardExtensionDiscoveringManager;
+import org.springframework.beans.factory.FactoryBean;
+
+/**
+ * Spring factory bean that returns the ExtensionManager instance from ExtensionManagerHolder.
+ *
+ * The ExtensionManagerHolder will be initialized before the Spring context starts.
+ */
+public class ExtensionManagerFactoryBean implements FactoryBean<ExtensionManager> {
+
+    @Override
+    public ExtensionManager getObject() {
+        return ExtensionManagerHolder.getExtensionManager();
+    }
+
+    @Override
+    public Class<?> getObjectType() {
+        return StandardExtensionDiscoveringManager.class;
+    }
+
+    @Override
+    public boolean isSingleton() {
+        return true;
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/fdd8cdbb/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/spring/FlowControllerFactoryBean.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/spring/FlowControllerFactoryBean.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/spring/FlowControllerFactoryBean.java
index c7a7e7d..2124daf 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/spring/FlowControllerFactoryBean.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/spring/FlowControllerFactoryBean.java
@@ -25,6 +25,7 @@ import org.apache.nifi.controller.FlowController;
 import org.apache.nifi.controller.leader.election.LeaderElectionManager;
 import org.apache.nifi.controller.repository.FlowFileEventRepository;
 import org.apache.nifi.encrypt.StringEncryptor;
+import org.apache.nifi.nar.ExtensionManager;
 import org.apache.nifi.registry.VariableRegistry;
 import org.apache.nifi.registry.flow.FlowRegistryClient;
 import org.apache.nifi.reporting.BulletinRepository;
@@ -51,6 +52,7 @@ public class FlowControllerFactoryBean implements FactoryBean, ApplicationContex
     private VariableRegistry variableRegistry;
     private LeaderElectionManager leaderElectionManager;
     private FlowRegistryClient flowRegistryClient;
+    private ExtensionManager extensionManager;
 
     @Override
     public Object getObject() throws Exception {
@@ -72,7 +74,8 @@ public class FlowControllerFactoryBean implements FactoryBean, ApplicationContex
                     heartbeatMonitor,
                     leaderElectionManager,
                     variableRegistry,
-                    flowRegistryClient);
+                    flowRegistryClient,
+                    extensionManager);
             } else {
                 flowController = FlowController.createStandaloneInstance(
                     flowFileEventRepository,
@@ -82,7 +85,8 @@ public class FlowControllerFactoryBean implements FactoryBean, ApplicationContex
                     encryptor,
                     bulletinRepository,
                     variableRegistry,
-                    flowRegistryClient);
+                    flowRegistryClient,
+                    extensionManager);
             }
 
         }
@@ -142,4 +146,8 @@ public class FlowControllerFactoryBean implements FactoryBean, ApplicationContex
     public void setFlowRegistryClient(final FlowRegistryClient flowRegistryClient) {
         this.flowRegistryClient = flowRegistryClient;
     }
+
+    public void setExtensionManager(ExtensionManager extensionManager) {
+        this.extensionManager = extensionManager;
+    }
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/fdd8cdbb/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/util/BundleUtils.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/util/BundleUtils.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/util/BundleUtils.java
index 5474a56..ab89c25 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/util/BundleUtils.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/util/BundleUtils.java
@@ -29,8 +29,8 @@ import java.util.stream.Collectors;
  * Utility class for Bundles.
  */
 public final class BundleUtils {
-    private static BundleCoordinate findBundleForType(final String type, final BundleCoordinate desiredCoordinate) {
-        final List<Bundle> bundles = ExtensionManager.getBundles(type);
+    private static BundleCoordinate findBundleForType(final ExtensionManager extensionManager, final String type, final BundleCoordinate desiredCoordinate) {
+        final List<Bundle> bundles = extensionManager.getBundles(type);
         if (bundles.isEmpty()) {
             throw new IllegalStateException(String.format("%s is not known to this NiFi instance.", type));
         } else if (bundles.size() > 1) {
@@ -44,18 +44,19 @@ public final class BundleUtils {
         }
     }
 
-    private static BundleCoordinate findCompatibleBundle(final String type, final BundleDTO bundleDTO, final boolean allowCompatibleBundle) {
+    private static BundleCoordinate findCompatibleBundle(final ExtensionManager extensionManager, final String type,
+                                                         final BundleDTO bundleDTO, final boolean allowCompatibleBundle) {
         final BundleCoordinate coordinate = new BundleCoordinate(bundleDTO.getGroup(), bundleDTO.getArtifact(), bundleDTO.getVersion());
-        final Bundle bundle = ExtensionManager.getBundle(coordinate);
+        final Bundle bundle = extensionManager.getBundle(coordinate);
 
         if (bundle == null) {
             if (allowCompatibleBundle) {
-                return findBundleForType(type, coordinate);
+                return findBundleForType(extensionManager, type, coordinate);
             } else {
                 throw new IllegalStateException(String.format("%s from %s is not known to this NiFi instance.", type, coordinate));
             }
         } else {
-            final List<BundleCoordinate> bundlesForType = ExtensionManager.getBundles(type).stream().map(b -> b.getBundleDetails().getCoordinate()).collect(Collectors.toList());
+            final List<BundleCoordinate> bundlesForType = extensionManager.getBundles(type).stream().map(b -> b.getBundleDetails().getCoordinate()).collect(Collectors.toList());
             if (bundlesForType.contains(coordinate)) {
                 return coordinate;
             } else {
@@ -91,11 +92,11 @@ public final class BundleUtils {
      * @return the bundle coordinate
      * @throws IllegalStateException bundle not found
      */
-    public static BundleCoordinate getBundle(final String type, final BundleDTO bundleDTO) {
+    public static BundleCoordinate getBundle(final ExtensionManager extensionManager, final String type, final BundleDTO bundleDTO) {
         if (bundleDTO == null) {
-            return findBundleForType(type, null);
+            return findBundleForType(extensionManager, type, null);
         } else {
-            return findCompatibleBundle(type, bundleDTO, false);
+            return findCompatibleBundle(extensionManager, type, bundleDTO, false);
         }
     }
 
@@ -132,11 +133,11 @@ public final class BundleUtils {
      * @return the bundle coordinate
      * @throws IllegalStateException no compatible bundle found
      */
-    public static BundleCoordinate getCompatibleBundle(final String type, final BundleDTO bundleDTO) {
+    public static BundleCoordinate getCompatibleBundle(final ExtensionManager extensionManager, final String type, final BundleDTO bundleDTO) {
         if (bundleDTO == null) {
-            return findBundleForType(type, null);
+            return findBundleForType(extensionManager, type, null);
         } else {
-            return findCompatibleBundle(type, bundleDTO, true);
+            return findCompatibleBundle(extensionManager, type, bundleDTO, true);
         }
     }
 
@@ -147,10 +148,10 @@ public final class BundleUtils {
      *
      * @param versionedGroup the versioned group
      */
-    public static void discoverCompatibleBundles(final VersionedProcessGroup versionedGroup) {
+    public static void discoverCompatibleBundles(final ExtensionManager extensionManager, final VersionedProcessGroup versionedGroup) {
         if (versionedGroup.getProcessors() != null) {
             versionedGroup.getProcessors().forEach(processor -> {
-                final BundleCoordinate coordinate = BundleUtils.getCompatibleBundle(processor.getType(), createBundleDto(processor.getBundle()));
+                final BundleCoordinate coordinate = BundleUtils.getCompatibleBundle(extensionManager, processor.getType(), createBundleDto(processor.getBundle()));
 
                 final org.apache.nifi.registry.flow.Bundle bundle = new org.apache.nifi.registry.flow.Bundle();
                 bundle.setArtifact(coordinate.getId());
@@ -162,7 +163,7 @@ public final class BundleUtils {
 
         if (versionedGroup.getControllerServices() != null) {
             versionedGroup.getControllerServices().forEach(controllerService -> {
-                final BundleCoordinate coordinate = BundleUtils.getCompatibleBundle(controllerService.getType(), createBundleDto(controllerService.getBundle()));
+                final BundleCoordinate coordinate = BundleUtils.getCompatibleBundle(extensionManager, controllerService.getType(), createBundleDto(controllerService.getBundle()));
 
                 final org.apache.nifi.registry.flow.Bundle bundle = new org.apache.nifi.registry.flow.Bundle();
                 bundle.setArtifact(coordinate.getId());
@@ -174,7 +175,7 @@ public final class BundleUtils {
 
         if (versionedGroup.getProcessGroups() != null) {
             versionedGroup.getProcessGroups().forEach(processGroup -> {
-                discoverCompatibleBundles(processGroup);
+                discoverCompatibleBundles(extensionManager, processGroup);
             });
         }
     }

http://git-wip-us.apache.org/repos/asf/nifi/blob/fdd8cdbb/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/resources/nifi-context.xml
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/resources/nifi-context.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/resources/nifi-context.xml
index d9f89aa..2261ae8 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/resources/nifi-context.xml
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/resources/nifi-context.xml
@@ -40,6 +40,10 @@
         <property name="properties" ref="nifiProperties" />
     </bean>
 
+    <!-- extension manager -->
+    <bean id="extensionManager" class="org.apache.nifi.spring.ExtensionManagerFactoryBean">
+    </bean>
+
     <!-- flow controller -->
     <bean id="flowController" class="org.apache.nifi.spring.FlowControllerFactoryBean">
         <property name="properties" ref="nifiProperties"/>
@@ -51,6 +55,7 @@
         <property name="variableRegistry" ref="variableRegistry"/>
         <property name="leaderElectionManager" ref="leaderElectionManager" />
         <property name="flowRegistryClient" ref="flowRegistryClient" />
+        <property name="extensionManager" ref="extensionManager" />
     </bean>
 
     <!-- flow service -->

http://git-wip-us.apache.org/repos/asf/nifi/blob/fdd8cdbb/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/StandardFlowServiceTest.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/StandardFlowServiceTest.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/StandardFlowServiceTest.java
index 4750746..df8a126 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/StandardFlowServiceTest.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/StandardFlowServiceTest.java
@@ -26,6 +26,8 @@ import org.apache.nifi.controller.serialization.ScheduledStateLookup;
 import org.apache.nifi.controller.serialization.StandardFlowSerializer;
 import org.apache.nifi.encrypt.StringEncryptor;
 import org.apache.nifi.events.VolatileBulletinRepository;
+import org.apache.nifi.nar.ExtensionDiscoveringManager;
+import org.apache.nifi.nar.ExtensionManager;
 import org.apache.nifi.registry.VariableRegistry;
 import org.apache.nifi.registry.flow.FlowRegistryClient;
 import org.apache.nifi.registry.variable.FileBasedVariableRegistry;
@@ -69,6 +71,7 @@ public class StandardFlowServiceTest {
     private StringEncryptor mockEncryptor;
     private RevisionManager revisionManager;
     private VariableRegistry variableRegistry;
+    private ExtensionManager extensionManager;
 
     @BeforeClass
     public static void setupSuite() {
@@ -86,8 +89,9 @@ public class StandardFlowServiceTest {
         authorizer = mock(Authorizer.class);
         mockAuditService = mock(AuditService.class);
         revisionManager = mock(RevisionManager.class);
+        extensionManager = mock(ExtensionDiscoveringManager.class);
         flowController = FlowController.createStandaloneInstance(mockFlowFileEventRepository, properties, authorizer, mockAuditService, mockEncryptor,
-                                        new VolatileBulletinRepository(), variableRegistry, mock(FlowRegistryClient.class));
+                                        new VolatileBulletinRepository(), variableRegistry, mock(FlowRegistryClient.class), extensionManager);
         flowService = StandardFlowService.createStandaloneInstance(flowController, properties, mockEncryptor, revisionManager, authorizer);
     }
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/fdd8cdbb/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 8291634..5516547 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
@@ -16,34 +16,6 @@
  */
 package org.apache.nifi.controller;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNotEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertTrue;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
-
-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.net.URLClassLoader;
-import java.nio.charset.StandardCharsets;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.LinkedHashMap;
-import java.util.LinkedHashSet;
-import java.util.Map;
-import java.util.Set;
-import java.util.UUID;
-import java.util.concurrent.TimeUnit;
 import org.apache.commons.io.FileUtils;
 import org.apache.commons.io.IOUtils;
 import org.apache.nifi.admin.service.AuditService;
@@ -72,8 +44,9 @@ import org.apache.nifi.groups.ProcessGroup;
 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.ExtensionDiscoveringManager;
 import org.apache.nifi.nar.InstanceClassLoader;
+import org.apache.nifi.nar.StandardExtensionDiscoveringManager;
 import org.apache.nifi.nar.SystemBundle;
 import org.apache.nifi.processor.Relationship;
 import org.apache.nifi.provenance.MockProvenanceRepository;
@@ -96,6 +69,35 @@ import org.junit.Before;
 import org.junit.Test;
 import org.mockito.Mockito;
 
+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.net.URLClassLoader;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.LinkedHashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.TimeUnit;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
 public class TestFlowController {
 
     private FlowController controller;
@@ -108,6 +110,7 @@ public class TestFlowController {
     private BulletinRepository bulletinRepo;
     private VariableRegistry variableRegistry;
     private volatile String propsFile = "src/test/resources/flowcontrollertest.nifi.properties";
+    private ExtensionDiscoveringManager extensionManager;
 
     /**
      * Utility method which accepts {@link NiFiProperties} object but calls {@link StringEncryptor#createEncryptor(String, String, String)} with extracted properties.
@@ -136,7 +139,8 @@ public class TestFlowController {
 
         // use the system bundle
         systemBundle = SystemBundle.create(nifiProperties);
-        ExtensionManager.discoverExtensions(systemBundle, Collections.emptySet());
+        extensionManager = new StandardExtensionDiscoveringManager();
+        extensionManager.discoverExtensions(systemBundle, Collections.emptySet());
 
         User user1 = new User.Builder().identifier("user-id-1").identity("user-1").build();
         User user2 = new User.Builder().identifier("user-id-2").identity("user-2").build();
@@ -179,7 +183,7 @@ public class TestFlowController {
 
         bulletinRepo = Mockito.mock(BulletinRepository.class);
         controller = FlowController.createStandaloneInstance(flowFileEventRepo, nifiProperties, authorizer,
-            auditService, encryptor, bulletinRepo, variableRegistry, Mockito.mock(FlowRegistryClient.class));
+            auditService, encryptor, bulletinRepo, variableRegistry, Mockito.mock(FlowRegistryClient.class), extensionManager);
     }
 
     @After
@@ -190,7 +194,8 @@ public class TestFlowController {
 
     @Test
     public void testSynchronizeFlowWithReportingTaskAndProcessorReferencingControllerService() throws IOException {
-        final FlowSynchronizer standardFlowSynchronizer = new StandardFlowSynchronizer(createEncryptorFromProperties(nifiProperties), nifiProperties);
+        final FlowSynchronizer standardFlowSynchronizer = new StandardFlowSynchronizer(
+                createEncryptorFromProperties(nifiProperties), nifiProperties, extensionManager);
 
         // create a mock proposed data flow with the same auth fingerprint as the current authorizer
         final String authFingerprint = authorizer.getFingerprint();
@@ -253,7 +258,8 @@ public class TestFlowController {
 
     @Test
     public void testSynchronizeFlowWithProcessorReferencingControllerService() throws IOException {
-        final FlowSynchronizer standardFlowSynchronizer = new StandardFlowSynchronizer(createEncryptorFromProperties(nifiProperties), nifiProperties);
+        final FlowSynchronizer standardFlowSynchronizer = new StandardFlowSynchronizer(
+                createEncryptorFromProperties(nifiProperties), nifiProperties, extensionManager);
 
         // create a mock proposed data flow with the same auth fingerprint as the current authorizer
         final String authFingerprint = authorizer.getFingerprint();
@@ -292,7 +298,8 @@ public class TestFlowController {
 
     @Test
     public void testSynchronizeFlowWhenAuthorizationsAreEqual() {
-        final FlowSynchronizer standardFlowSynchronizer = new StandardFlowSynchronizer(createEncryptorFromProperties(nifiProperties), nifiProperties);
+        final FlowSynchronizer standardFlowSynchronizer = new StandardFlowSynchronizer(
+                createEncryptorFromProperties(nifiProperties), nifiProperties, extensionManager);
 
         // create a mock proposed data flow with the same auth fingerprint as the current authorizer
         final String authFingerprint = authorizer.getFingerprint();
@@ -306,7 +313,8 @@ public class TestFlowController {
 
     @Test(expected = UninheritableFlowException.class)
     public void testSynchronizeFlowWhenAuthorizationsAreDifferent() {
-        final FlowSynchronizer standardFlowSynchronizer = new StandardFlowSynchronizer(createEncryptorFromProperties(nifiProperties), nifiProperties);
+        final FlowSynchronizer standardFlowSynchronizer = new StandardFlowSynchronizer(
+                createEncryptorFromProperties(nifiProperties), nifiProperties, extensionManager);
 
         // create a mock proposed data flow with different auth fingerprint as the current authorizer
         final String authFingerprint = "<authorizations></authorizations>";
@@ -319,7 +327,8 @@ public class TestFlowController {
 
     @Test(expected = UninheritableFlowException.class)
     public void testSynchronizeFlowWhenProposedAuthorizationsAreNull() {
-        final FlowSynchronizer standardFlowSynchronizer = new StandardFlowSynchronizer(createEncryptorFromProperties(nifiProperties), nifiProperties);
+        final FlowSynchronizer standardFlowSynchronizer = new StandardFlowSynchronizer(
+                createEncryptorFromProperties(nifiProperties), nifiProperties, extensionManager);
 
         final DataFlow proposedDataFlow = Mockito.mock(DataFlow.class);
         when(proposedDataFlow.getAuthorizerFingerprint()).thenReturn(null);
@@ -329,7 +338,8 @@ public class TestFlowController {
 
     @Test
     public void testSynchronizeFlowWhenCurrentAuthorizationsAreEmptyAndProposedAreNot() {
-        final FlowSynchronizer standardFlowSynchronizer = new StandardFlowSynchronizer(createEncryptorFromProperties(nifiProperties), nifiProperties);
+        final FlowSynchronizer standardFlowSynchronizer = new StandardFlowSynchronizer(
+                createEncryptorFromProperties(nifiProperties), nifiProperties, extensionManager);
 
         // create a mock proposed data flow with the same auth fingerprint as the current authorizer
         final String authFingerprint = authorizer.getFingerprint();
@@ -341,14 +351,15 @@ public class TestFlowController {
 
         controller.shutdown(true);
         controller = FlowController.createStandaloneInstance(flowFileEventRepo, nifiProperties, authorizer,
-            auditService, encryptor, bulletinRepo, variableRegistry, Mockito.mock(FlowRegistryClient.class));
+            auditService, encryptor, bulletinRepo, variableRegistry, Mockito.mock(FlowRegistryClient.class), extensionManager);
         controller.synchronize(standardFlowSynchronizer, proposedDataFlow);
         assertEquals(authFingerprint, authorizer.getFingerprint());
     }
 
     @Test
     public void testSynchronizeFlowWhenProposedMissingComponentsAreDifferent() {
-        final FlowSynchronizer standardFlowSynchronizer = new StandardFlowSynchronizer(createEncryptorFromProperties(nifiProperties), nifiProperties);
+        final FlowSynchronizer standardFlowSynchronizer = new StandardFlowSynchronizer(
+                createEncryptorFromProperties(nifiProperties), nifiProperties, extensionManager);
 
         final Set<String> missingComponents = new HashSet<>();
         missingComponents.add("1");
@@ -368,7 +379,7 @@ public class TestFlowController {
     @Test
     public void testSynchronizeFlowWhenExistingMissingComponentsAreDifferent() throws IOException {
         final StringEncryptor stringEncryptor = createEncryptorFromProperties(nifiProperties);
-        final FlowSynchronizer standardFlowSynchronizer = new StandardFlowSynchronizer(stringEncryptor, nifiProperties);
+        final FlowSynchronizer standardFlowSynchronizer = new StandardFlowSynchronizer(stringEncryptor, nifiProperties, extensionManager);
 
         final ProcessorNode mockProcessorNode = mock(ProcessorNode.class);
         when(mockProcessorNode.getIdentifier()).thenReturn("1");
@@ -408,7 +419,8 @@ public class TestFlowController {
 
     @Test
     public void testSynchronizeFlowWhenBundlesAreSame() throws IOException {
-        final FlowSynchronizer standardFlowSynchronizer = new StandardFlowSynchronizer(createEncryptorFromProperties(nifiProperties), nifiProperties);
+        final FlowSynchronizer standardFlowSynchronizer = new StandardFlowSynchronizer(
+                createEncryptorFromProperties(nifiProperties), nifiProperties, extensionManager);
 
         final LogRepository logRepository = LogRepositoryFactory.getRepository("d89ada5d-35fb-44ff-83f1-4cc00b48b2df");
         logRepository.removeAllObservers();
@@ -419,7 +431,8 @@ public class TestFlowController {
 
     @Test
     public void testSynchronizeFlowWhenBundlesAreDifferent() throws IOException {
-        final FlowSynchronizer standardFlowSynchronizer = new StandardFlowSynchronizer(createEncryptorFromProperties(nifiProperties), nifiProperties);
+        final FlowSynchronizer standardFlowSynchronizer = new StandardFlowSynchronizer(
+                createEncryptorFromProperties(nifiProperties), nifiProperties, extensionManager);
 
         final LogRepository logRepository = LogRepositoryFactory.getRepository("d89ada5d-35fb-44ff-83f1-4cc00b48b2df");
         logRepository.removeAllObservers();
@@ -615,7 +628,7 @@ public class TestFlowController {
         final String originalName = processorNode.getName();
 
         // the instance class loader shouldn't have any of the resources yet
-        InstanceClassLoader instanceClassLoader = ExtensionManager.getInstanceClassLoader(id);
+        InstanceClassLoader instanceClassLoader = extensionManager.getInstanceClassLoader(id);
         assertNotNull(instanceClassLoader);
         assertFalse(containsResource(instanceClassLoader.getURLs(), resource1));
         assertFalse(containsResource(instanceClassLoader.getURLs(), resource2));
@@ -626,7 +639,7 @@ public class TestFlowController {
         controller.reload(processorNode, DummySettingsProcessor.class.getName(), coordinate, additionalUrls);
 
         // the instance class loader shouldn't have any of the resources yet
-        instanceClassLoader = ExtensionManager.getInstanceClassLoader(id);
+        instanceClassLoader = extensionManager.getInstanceClassLoader(id);
         assertNotNull(instanceClassLoader);
         assertTrue(containsResource(instanceClassLoader.getURLs(), resource1));
         assertTrue(containsResource(instanceClassLoader.getURLs(), resource2));
@@ -676,7 +689,7 @@ public class TestFlowController {
         final ControllerServiceNode controllerServiceNode = controller.createControllerService(ServiceA.class.getName(), id, coordinate, null, true);
 
         // the instance class loader shouldn't have any of the resources yet
-        URLClassLoader instanceClassLoader = ExtensionManager.getInstanceClassLoader(id);
+        URLClassLoader instanceClassLoader = extensionManager.getInstanceClassLoader(id);
         assertNotNull(instanceClassLoader);
         assertFalse(containsResource(instanceClassLoader.getURLs(), resource1));
         assertFalse(containsResource(instanceClassLoader.getURLs(), resource2));
@@ -687,7 +700,7 @@ public class TestFlowController {
         controller.reload(controllerServiceNode, ServiceB.class.getName(), coordinate, additionalUrls);
 
         // the instance class loader shouldn't have any of the resources yet
-        instanceClassLoader = ExtensionManager.getInstanceClassLoader(id);
+        instanceClassLoader = extensionManager.getInstanceClassLoader(id);
         assertNotNull(instanceClassLoader);
         assertTrue(containsResource(instanceClassLoader.getURLs(), resource1));
         assertTrue(containsResource(instanceClassLoader.getURLs(), resource2));
@@ -738,7 +751,7 @@ public class TestFlowController {
         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);
+        InstanceClassLoader instanceClassLoader = extensionManager.getInstanceClassLoader(id);
         assertNotNull(instanceClassLoader);
         assertFalse(containsResource(instanceClassLoader.getURLs(), resource1));
         assertFalse(containsResource(instanceClassLoader.getURLs(), resource2));
@@ -748,7 +761,7 @@ public class TestFlowController {
         controller.reload(node, DummyScheduledReportingTask.class.getName(), coordinate, additionalUrls);
 
         // the instance class loader shouldn't have any of the resources yet
-        instanceClassLoader = ExtensionManager.getInstanceClassLoader(id);
+        instanceClassLoader = extensionManager.getInstanceClassLoader(id);
         assertNotNull(instanceClassLoader);
         assertTrue(containsResource(instanceClassLoader.getURLs(), resource1));
         assertTrue(containsResource(instanceClassLoader.getURLs(), resource2));

http://git-wip-us.apache.org/repos/asf/nifi/blob/fdd8cdbb/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 eabc899..d2be387 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
@@ -17,26 +17,6 @@
 
 package org.apache.nifi.controller;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-import java.io.File;
-import java.net.MalformedURLException;
-import java.net.URL;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.LinkedHashMap;
-import java.util.LinkedHashSet;
-import java.util.Map;
-import java.util.Set;
-import java.util.UUID;
-import java.util.concurrent.Callable;
-import java.util.concurrent.Future;
-import java.util.concurrent.ScheduledExecutorService;
-
 import org.apache.nifi.annotation.lifecycle.OnScheduled;
 import org.apache.nifi.annotation.lifecycle.OnStopped;
 import org.apache.nifi.annotation.lifecycle.OnUnscheduled;
@@ -51,8 +31,9 @@ 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.nar.ExtensionManager;
+import org.apache.nifi.nar.ExtensionDiscoveringManager;
 import org.apache.nifi.nar.NarCloseable;
+import org.apache.nifi.nar.StandardExtensionDiscoveringManager;
 import org.apache.nifi.nar.SystemBundle;
 import org.apache.nifi.processor.AbstractProcessor;
 import org.apache.nifi.processor.ProcessContext;
@@ -77,13 +58,41 @@ import org.junit.Before;
 import org.junit.Test;
 import org.mockito.Mockito;
 
+import java.io.File;
+import java.net.MalformedURLException;
+import java.net.URL;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.LinkedHashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.Callable;
+import java.util.concurrent.Future;
+import java.util.concurrent.ScheduledExecutorService;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
 public class TestStandardProcessorNode {
 
     private MockVariableRegistry variableRegistry;
+    private Bundle systemBundle;
+    private ExtensionDiscoveringManager extensionManager;
+    private NiFiProperties niFiProperties;
 
     @Before
     public void setup() {
         variableRegistry = new MockVariableRegistry();
+        niFiProperties = NiFiProperties.createBasicNiFiProperties("src/test/resources/conf/nifi.properties", null);
+
+        systemBundle = SystemBundle.create(niFiProperties);
+        extensionManager = new StandardExtensionDiscoveringManager();
+        extensionManager.discoverExtensions(systemBundle, Collections.emptySet());
     }
 
     @Test(timeout = 10000)
@@ -99,7 +108,7 @@ public class TestStandardProcessorNode {
 
         final LoggableComponent<Processor> loggableComponent = new LoggableComponent<>(processor, coordinate, null);
         final StandardProcessorNode procNode = new StandardProcessorNode(loggableComponent, uuid, createValidationContextFactory(), null, null,
-            NiFiProperties.createBasicNiFiProperties(null, null), new StandardComponentVariableRegistry(VariableRegistry.EMPTY_REGISTRY), reloadComponent, new SynchronousValidationTrigger());
+            niFiProperties, new StandardComponentVariableRegistry(VariableRegistry.EMPTY_REGISTRY), reloadComponent, extensionManager, new SynchronousValidationTrigger());
         final ScheduledExecutorService taskScheduler = new FlowEngine(1, "TestClasspathResources", true);
 
         final StandardProcessContext processContext = new StandardProcessContext(procNode, null, null, null, () -> false);
@@ -138,7 +147,7 @@ public class TestStandardProcessorNode {
         final ModifiesClasspathProcessor processor = new ModifiesClasspathProcessor(Arrays.asList(classpathProp));
         final StandardProcessorNode procNode = createProcessorNode(processor, reloadComponent);
 
-        try (final NarCloseable narCloseable = NarCloseable.withComponentNarLoader(procNode.getProcessor().getClass(), procNode.getIdentifier())){
+        try (final NarCloseable narCloseable = NarCloseable.withComponentNarLoader(extensionManager, procNode.getProcessor().getClass(), procNode.getIdentifier())){
 
             // Should not have any of the test resources loaded at this point
             final URL[] testResources = getTestResources();
@@ -163,7 +172,7 @@ public class TestStandardProcessorNode {
             // Should pass validation
             assertTrue(procNode.computeValidationErrors(procNode.getValidationContext()).isEmpty());
         } finally {
-            ExtensionManager.removeInstanceClassLoader(procNode.getIdentifier());
+            extensionManager.removeInstanceClassLoader(procNode.getIdentifier());
         }
     }
 
@@ -180,7 +189,7 @@ public class TestStandardProcessorNode {
         final ModifiesClasspathProcessor processor = new ModifiesClasspathProcessor(Arrays.asList(classpathProp, otherProp));
         final StandardProcessorNode procNode = createProcessorNode(processor, reloadComponent);
 
-        try (final NarCloseable narCloseable = NarCloseable.withComponentNarLoader(procNode.getProcessor().getClass(), procNode.getIdentifier())){
+        try (final NarCloseable narCloseable = NarCloseable.withComponentNarLoader(extensionManager, procNode.getProcessor().getClass(), procNode.getIdentifier())){
             // Should not have any of the test resources loaded at this point
             final URL[] testResources = getTestResources();
             for (URL testResource : testResources) {
@@ -230,7 +239,7 @@ public class TestStandardProcessorNode {
             // Should STILL pass validation
             assertTrue(procNode.computeValidationErrors(procNode.getValidationContext()).isEmpty());
         } finally {
-            ExtensionManager.removeInstanceClassLoader(procNode.getIdentifier());
+            extensionManager.removeInstanceClassLoader(procNode.getIdentifier());
         }
     }
 
@@ -246,7 +255,7 @@ public class TestStandardProcessorNode {
         final ModifiesClasspathProcessor processor = new ModifiesClasspathProcessor(Arrays.asList(classpathProp1, classpathProp2));
         final StandardProcessorNode procNode = createProcessorNode(processor, reloadComponent);
 
-        try (final NarCloseable narCloseable = NarCloseable.withComponentNarLoader(procNode.getProcessor().getClass(), procNode.getIdentifier())){
+        try (final NarCloseable narCloseable = NarCloseable.withComponentNarLoader(extensionManager, procNode.getProcessor().getClass(), procNode.getIdentifier())){
 
             // Should not have any of the test resources loaded at this point
             final URL[] testResources = getTestResources();
@@ -275,7 +284,7 @@ public class TestStandardProcessorNode {
             // Should pass validation
             assertTrue(procNode.computeValidationErrors(procNode.getValidationContext()).isEmpty());
         } finally {
-            ExtensionManager.removeInstanceClassLoader(procNode.getIdentifier());
+            extensionManager.removeInstanceClassLoader(procNode.getIdentifier());
         }
     }
 
@@ -291,7 +300,7 @@ public class TestStandardProcessorNode {
         final ModifiesClasspathProcessor processor = new ModifiesClasspathProcessor(Arrays.asList(classpathProp1, classpathProp2));
         final StandardProcessorNode procNode = createProcessorNode(processor, reloadComponent);
 
-        try (final NarCloseable narCloseable = NarCloseable.withComponentNarLoader(procNode.getProcessor().getClass(), procNode.getIdentifier())){
+        try (final NarCloseable narCloseable = NarCloseable.withComponentNarLoader(extensionManager, procNode.getProcessor().getClass(), procNode.getIdentifier())){
 
             // Should not have any of the test resources loaded at this point
             final URL[] testResources = getTestResources();
@@ -317,7 +326,7 @@ public class TestStandardProcessorNode {
             // Should pass validation
             assertTrue(procNode.computeValidationErrors(procNode.getValidationContext()).isEmpty());
         } finally {
-            ExtensionManager.removeInstanceClassLoader(procNode.getIdentifier());
+            extensionManager.removeInstanceClassLoader(procNode.getIdentifier());
         }
     }
 
@@ -327,7 +336,7 @@ public class TestStandardProcessorNode {
         final ModifiesClasspathNoAnnotationProcessor processor = new ModifiesClasspathNoAnnotationProcessor();
         final StandardProcessorNode procNode = createProcessorNode(processor, reloadComponent);
 
-        try (final NarCloseable narCloseable = NarCloseable.withComponentNarLoader(procNode.getProcessor().getClass(), procNode.getIdentifier())){
+        try (final NarCloseable narCloseable = NarCloseable.withComponentNarLoader(extensionManager, procNode.getProcessor().getClass(), procNode.getIdentifier())){
 
             final Map<String, String> properties = new HashMap<>();
             properties.put(ModifiesClasspathNoAnnotationProcessor.CLASSPATH_RESOURCE.getName(),
@@ -344,7 +353,7 @@ public class TestStandardProcessorNode {
             // Should pass validation
             assertTrue(procNode.computeValidationErrors(procNode.getValidationContext()).isEmpty());
         } finally {
-            ExtensionManager.removeInstanceClassLoader(procNode.getIdentifier());
+            extensionManager.removeInstanceClassLoader(procNode.getIdentifier());
         }
     }
 
@@ -387,20 +396,18 @@ public class TestStandardProcessorNode {
     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 TerminationAwareLogger componentLog = Mockito.mock(TerminationAwareLogger.class);
 
-        final Bundle systemBundle = SystemBundle.create(niFiProperties);
-        ExtensionManager.discoverExtensions(systemBundle, Collections.emptySet());
-        ExtensionManager.createInstanceClassLoader(processor.getClass().getName(), uuid, systemBundle, null);
+        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, new StandardComponentVariableRegistry(variableRegistry), reloadComponent, new SynchronousValidationTrigger());
+            null, niFiProperties, new StandardComponentVariableRegistry(variableRegistry), reloadComponent, extensionManager,
+                new SynchronousValidationTrigger());
     }
 
     private static class MockReloadComponent implements ReloadComponent {

http://git-wip-us.apache.org/repos/asf/nifi/blob/fdd8cdbb/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/reporting/TestStandardReportingContext.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/reporting/TestStandardReportingContext.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/reporting/TestStandardReportingContext.java
index f4606cc..e7f3714 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/reporting/TestStandardReportingContext.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/reporting/TestStandardReportingContext.java
@@ -16,14 +16,6 @@
  */
 package org.apache.nifi.controller.reporting;
 
-import static org.junit.Assert.assertEquals;
-
-import java.io.File;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.LinkedHashSet;
-import java.util.Map;
-import java.util.Set;
 import org.apache.commons.io.FileUtils;
 import org.apache.nifi.admin.service.AuditService;
 import org.apache.nifi.authorization.AbstractPolicyBasedAuthorizer;
@@ -40,7 +32,8 @@ import org.apache.nifi.controller.FlowController;
 import org.apache.nifi.controller.ReportingTaskNode;
 import org.apache.nifi.controller.repository.FlowFileEventRepository;
 import org.apache.nifi.encrypt.StringEncryptor;
-import org.apache.nifi.nar.ExtensionManager;
+import org.apache.nifi.nar.ExtensionDiscoveringManager;
+import org.apache.nifi.nar.StandardExtensionDiscoveringManager;
 import org.apache.nifi.nar.SystemBundle;
 import org.apache.nifi.provenance.MockProvenanceRepository;
 import org.apache.nifi.registry.VariableRegistry;
@@ -54,6 +47,15 @@ import org.junit.Before;
 import org.junit.Test;
 import org.mockito.Mockito;
 
+import java.io.File;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedHashSet;
+import java.util.Map;
+import java.util.Set;
+
+import static org.junit.Assert.assertEquals;
+
 public class TestStandardReportingContext {
 
     private static final String DEFAULT_SENSITIVE_PROPS_KEY = "nififtw!";
@@ -64,6 +66,7 @@ public class TestStandardReportingContext {
     private StringEncryptor encryptor;
     private NiFiProperties nifiProperties;
     private Bundle systemBundle;
+    private ExtensionDiscoveringManager extensionManager;
     private BulletinRepository bulletinRepo;
     private VariableRegistry variableRegistry;
     private FlowRegistryClient flowRegistry;
@@ -89,7 +92,8 @@ public class TestStandardReportingContext {
 
         // use the system bundle
         systemBundle = SystemBundle.create(nifiProperties);
-        ExtensionManager.discoverExtensions(systemBundle, Collections.emptySet());
+        extensionManager = new StandardExtensionDiscoveringManager();
+        extensionManager.discoverExtensions(systemBundle, Collections.emptySet());
 
         User user1 = new User.Builder().identifier("user-id-1").identity("user-1").build();
         User user2 = new User.Builder().identifier("user-id-2").identity("user-2").build();
@@ -132,7 +136,8 @@ public class TestStandardReportingContext {
         flowRegistry = Mockito.mock(FlowRegistryClient.class);
 
         bulletinRepo = Mockito.mock(BulletinRepository.class);
-        controller = FlowController.createStandaloneInstance(flowFileEventRepo, nifiProperties, authorizer, auditService, encryptor, bulletinRepo, variableRegistry, flowRegistry);
+        controller = FlowController.createStandaloneInstance(flowFileEventRepo, nifiProperties, authorizer, auditService, encryptor,
+                bulletinRepo, variableRegistry, flowRegistry, extensionManager);
     }
 
     @After

http://git-wip-us.apache.org/repos/asf/nifi/blob/fdd8cdbb/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/scheduling/ProcessorLifecycleIT.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/scheduling/ProcessorLifecycleIT.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/scheduling/ProcessorLifecycleIT.java
index b8fed54..fa164fc 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/scheduling/ProcessorLifecycleIT.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/scheduling/ProcessorLifecycleIT.java
@@ -37,7 +37,8 @@ import org.apache.nifi.controller.repository.FlowFileEventRepository;
 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.ExtensionDiscoveringManager;
+import org.apache.nifi.nar.StandardExtensionDiscoveringManager;
 import org.apache.nifi.nar.SystemBundle;
 import org.apache.nifi.processor.AbstractProcessor;
 import org.apache.nifi.processor.ProcessContext;
@@ -654,12 +655,13 @@ public class ProcessorLifecycleIT {
         final NiFiProperties nifiProperties = NiFiProperties.createBasicNiFiProperties(propsFile, addProps);
 
         final Bundle systemBundle = SystemBundle.create(nifiProperties);
-        ExtensionManager.discoverExtensions(systemBundle, Collections.emptySet());
+        final ExtensionDiscoveringManager extensionManager = new StandardExtensionDiscoveringManager();
+        extensionManager.discoverExtensions(systemBundle, Collections.emptySet());
 
         final FlowController flowController = FlowController.createStandaloneInstance(mock(FlowFileEventRepository.class), nifiProperties,
                 mock(Authorizer.class), mock(AuditService.class), null, new VolatileBulletinRepository(),
             new FileBasedVariableRegistry(nifiProperties.getVariableRegistryPropertiesPaths()),
-            mock(FlowRegistryClient.class));
+            mock(FlowRegistryClient.class), extensionManager);
 
         return new FlowControllerAndSystemBundle(flowController, systemBundle);
     }


[3/7] nifi git commit: NIFI-5673 Set up property/assembly for new auto-load directory - Set up NarAutoLoader to watch directory for new files - Move NarAutoLoader to JettyServer since it will need access to ExtensionManager - Created NarLoader to shared

Posted by ma...@apache.org.
http://git-wip-us.apache.org/repos/asf/nifi/blob/fdd8cdbb/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
index 06fdead..c1ca26e 100644
--- 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
@@ -34,11 +34,17 @@ import org.apache.nifi.processor.ProcessorInitializationContext;
  */
 public class ProcessorInitializer implements ConfigurableComponentInitializer {
 
+    private final ExtensionManager extensionManager;
+
+    public ProcessorInitializer(final ExtensionManager extensionManager) {
+        this.extensionManager = extensionManager;
+    }
+
     @Override
     public void initialize(ConfigurableComponent component) {
         Processor processor = (Processor) component;
         ProcessorInitializationContext initializationContext = new MockProcessorInitializationContext();
-        try (NarCloseable narCloseable = NarCloseable.withComponentNarLoader(component.getClass(), initializationContext.getIdentifier())) {
+        try (NarCloseable narCloseable = NarCloseable.withComponentNarLoader(extensionManager, component.getClass(), initializationContext.getIdentifier())) {
             processor.initialize(initializationContext);
         }
     }
@@ -46,13 +52,13 @@ public class ProcessorInitializer implements ConfigurableComponentInitializer {
     @Override
     public void teardown(ConfigurableComponent component) {
         Processor processor = (Processor) component;
-        try (NarCloseable narCloseable = NarCloseable.withComponentNarLoader(component.getClass(), component.getIdentifier())) {
+        try (NarCloseable narCloseable = NarCloseable.withComponentNarLoader(extensionManager, 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());
+            extensionManager.removeInstanceClassLoader(component.getIdentifier());
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/fdd8cdbb/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
index f0f495d..fa2911e 100644
--- 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
@@ -34,11 +34,17 @@ import org.apache.nifi.reporting.ReportingTask;
  */
 public class ReportingTaskingInitializer implements ConfigurableComponentInitializer {
 
+    private final ExtensionManager extensionManager;
+
+    public ReportingTaskingInitializer(final ExtensionManager extensionManager) {
+        this.extensionManager = extensionManager;
+    }
+
     @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())) {
+        try (NarCloseable narCloseable = NarCloseable.withComponentNarLoader(extensionManager, component.getClass(), context.getIdentifier())) {
             reportingTask.initialize(context);
         }
     }
@@ -46,12 +52,12 @@ public class ReportingTaskingInitializer implements ConfigurableComponentInitial
     @Override
     public void teardown(ConfigurableComponent component) {
         ReportingTask reportingTask = (ReportingTask) component;
-        try (NarCloseable narCloseable = NarCloseable.withComponentNarLoader(component.getClass(), component.getIdentifier())) {
+        try (NarCloseable narCloseable = NarCloseable.withComponentNarLoader(extensionManager, component.getClass(), component.getIdentifier())) {
 
             final MockConfigurationContext context = new MockConfigurationContext();
             ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnShutdown.class, reportingTask, new MockComponentLogger(), context);
         } finally {
-            ExtensionManager.removeInstanceClassLoader(component.getIdentifier());
+            extensionManager.removeInstanceClassLoader(component.getIdentifier());
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/fdd8cdbb/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/nar/ExtensionDiscoveringManager.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/nar/ExtensionDiscoveringManager.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/nar/ExtensionDiscoveringManager.java
new file mode 100644
index 0000000..00b1258
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/nar/ExtensionDiscoveringManager.java
@@ -0,0 +1,47 @@
+/*
+ * 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 java.util.Set;
+
+/**
+ * ExtensionManage that can discovers and load extensions.
+ */
+public interface ExtensionDiscoveringManager extends ExtensionManager {
+
+    /**
+     * Discovers all extensions available in the provided bundles.
+     *
+     * This method is intended to be called only once during application start-up.
+     *
+     * @param systemBundle the system bundle
+     * @param narBundles the bundles to use for discovering extensions
+     */
+    void discoverExtensions(Bundle systemBundle, Set<Bundle> narBundles);
+
+    /**
+     * Discovers extensions in the provided bundles.
+     *
+     * This method is intended to be used to discover additional extensions after the application is running.
+     *
+     * @param narBundles the bundles to use for discovering extensions
+     */
+    void discoverExtensions(Set<Bundle> narBundles);
+
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/fdd8cdbb/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
index f4a59f8..89755d1 100644
--- 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
@@ -16,402 +16,36 @@
  */
 package org.apache.nifi.nar;
 
-import org.apache.nifi.annotation.behavior.RequiresInstanceClassLoading;
-import org.apache.nifi.authentication.LoginIdentityProvider;
-import org.apache.nifi.authorization.AccessPolicyProvider;
-import org.apache.nifi.authorization.Authorizer;
-import org.apache.nifi.authorization.UserGroupProvider;
 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
+ * Provides the framework with access to extensions, bundles, and class loaders.
  */
-@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(UserGroupProvider.class, new HashSet<>());
-        definitionMap.put(AccessPolicyProvider.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<>());
-    }
-
-    public static Set<Bundle> getAllBundles() {
-        return classNameBundleLookup.values().stream()
-            .flatMap(List::stream)
-            .collect(Collectors.toSet());
-    }
+public interface ExtensionManager {
 
     /**
-     * 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.
+     * Retrieves all bundles known to this ExtensionManager.
      *
-     * @param bundle from which to load extensions
+     * @return the set of known bundles
      */
-    @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();
-    }
+    Set<Bundle> getAllBundles();
 
     /**
-     * Registers extension for the specified type from the specified Bundle.
+     * Creates the ClassLoader for the instance of the given type.
      *
-     * @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 classType the type of class to create 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;
-    }
+    InstanceClassLoader createInstanceClassLoader(String classType, String instanceIdentifier, Bundle bundle, Set<URL> additionalUrls);
 
     /**
      * Retrieves the InstanceClassLoader for the component with the given identifier.
@@ -419,24 +53,14 @@ public class ExtensionManager {
      * @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);
-    }
+    InstanceClassLoader getInstanceClassLoader(String 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;
-    }
+    InstanceClassLoader removeInstanceClassLoader(String instanceIdentifier);
 
     /**
      * Closes the given ClassLoader if it is an instance of URLClassLoader.
@@ -444,16 +68,7 @@ public class ExtensionManager {
      * @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);
-            }
-        }
-    }
+    void closeURLClassLoader(String instanceIdentifier, ClassLoader classLoader);
 
     /**
      * Retrieves the bundles that have a class with the given name.
@@ -461,13 +76,7 @@ public class ExtensionManager {
      * @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);
-    }
+    List<Bundle> getBundles(String classType);
 
     /**
      * Retrieves the bundle with the given coordinate.
@@ -475,12 +84,15 @@ public class ExtensionManager {
      * @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);
-    }
+    Bundle getBundle(BundleCoordinate bundleCoordinate);
+
+    /**
+     * Retrieves the extension classes that were loaded from the bundle with the given coordinate.
+     *
+     * @param bundleCoordinate the coordinate
+     * @return the classes from the bundle with that coordinate
+     */
+    Set<Class> getTypes(BundleCoordinate bundleCoordinate);
 
     /**
      * Retrieves the bundle for the given class loader.
@@ -488,60 +100,30 @@ public class ExtensionManager {
      * @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();
+    Bundle getBundle(ClassLoader classLoader);
 
-        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());
+    /**
+     * Retrieves the set of classes that have been loaded for the definition.
+     *
+     * (i.e getExtensions(Processor.class)
+     *
+     * @param definition the extension definition, such as Processor.class
+     * @return the set of extensions implementing the defintion
+     */
+    Set<Class> getExtensions(Class<?> definition);
 
-                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);
-                }
-            }
+    /**
+     * Gets the temp component with the given type from the given bundle.
+     *
+     * @param classType the class name
+     * @param bundleCoordinate the coordinate
+     * @return the temp component instance
+     */
+    ConfigurableComponent getTempComponent(String classType, BundleCoordinate bundleCoordinate);
 
-            builder.append("\n\t=== End ").append(entry.getKey().getSimpleName()).append(" types ===");
-        }
+    /**
+     * Logs the available class loaders.
+     */
+    void logClassLoaderMapping();
 
-        logger.info(builder.toString());
-    }
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/fdd8cdbb/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/nar/ExtensionManagerHolder.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/nar/ExtensionManagerHolder.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/nar/ExtensionManagerHolder.java
new file mode 100644
index 0000000..f1a889f
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/nar/ExtensionManagerHolder.java
@@ -0,0 +1,61 @@
+/*
+ * 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;
+
+/**
+ * Holds a singleton instance of ExtensionManager.
+ *
+ * NOTE: This class primarily exists to create a bridge from the JettyServer to the Spring context. There
+ * should be no direct calls to this class outside of the JettyServer or the ExtensionManagerFactoryBean.
+ * The rest of the framework should obtain an instance of ExtensionManager from Spring.
+ */
+public final class ExtensionManagerHolder {
+
+    private static volatile ExtensionManager INSTANCE;
+
+    public static void init(final ExtensionManager extensionManager) {
+        if (INSTANCE == null) {
+            synchronized (ExtensionManagerHolder.class) {
+                if (INSTANCE == null) {
+                    INSTANCE = extensionManager;
+                } else {
+                    throw new IllegalStateException("Cannot reinitialize ExtensionManagerHolder");
+                }
+            }
+        } else {
+            throw new IllegalStateException("Cannot reinitialize ExtensionManagerHolder");
+        }
+    }
+
+    public static ExtensionManager getExtensionManager() {
+        if (INSTANCE == null) {
+            synchronized (ExtensionManagerHolder.class) {
+                if (INSTANCE == null) {
+                    throw new IllegalStateException("ExtensionManagerHolder was never initialized");
+                }
+            }
+        }
+
+        return INSTANCE;
+    }
+
+    // Private access
+    private ExtensionManagerHolder() {
+
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/fdd8cdbb/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
index 88d47ff..3e293cd 100644
--- 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
@@ -45,10 +45,10 @@ public class NarCloseable implements Closeable {
      * @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) {
+    public static NarCloseable withComponentNarLoader(final ExtensionManager extensionManager, final Class componentClass, final String componentIdentifier) {
         final ClassLoader current = Thread.currentThread().getContextClassLoader();
 
-        ClassLoader componentClassLoader = ExtensionManager.getInstanceClassLoader(componentIdentifier);
+        ClassLoader componentClassLoader = extensionManager.getInstanceClassLoader(componentIdentifier);
         if (componentClassLoader == null) {
             componentClassLoader = componentClass.getClassLoader();
         }
@@ -81,7 +81,7 @@ public class NarCloseable implements Closeable {
     public static NarCloseable withFrameworkNar() {
         final ClassLoader frameworkClassLoader;
         try {
-            frameworkClassLoader = NarClassLoaders.getInstance().getFrameworkBundle().getClassLoader();
+            frameworkClassLoader = NarClassLoadersHolder.getInstance().getFrameworkBundle().getClassLoader();
         } catch (final Exception e) {
             // This should never happen in a running instance, but it will occur in unit tests
             logger.error("Unable to access Framework ClassLoader due to " + e + ". Will continue without changing ClassLoaders.");

http://git-wip-us.apache.org/repos/asf/nifi/blob/fdd8cdbb/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
index 0be99dc..241bcd5 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/nar/NarThreadContextClassLoader.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/nar/NarThreadContextClassLoader.java
@@ -189,12 +189,12 @@ public class NarThreadContextClassLoader extends URLClassLoader {
      * @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)
+    public static <T> T createInstance(final ExtensionManager extensionManager, final String implementationClassName, final Class<T> typeDefinition, final NiFiProperties nifiProperties)
             throws InstantiationException, IllegalAccessException, ClassNotFoundException {
         final ClassLoader originalClassLoader = Thread.currentThread().getContextClassLoader();
         Thread.currentThread().setContextClassLoader(NarThreadContextClassLoader.getInstance());
         try {
-            final List<Bundle> bundles = ExtensionManager.getBundles(implementationClassName);
+            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));
             }

http://git-wip-us.apache.org/repos/asf/nifi/blob/fdd8cdbb/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/nar/StandardExtensionDiscoveringManager.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/nar/StandardExtensionDiscoveringManager.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/nar/StandardExtensionDiscoveringManager.java
new file mode 100644
index 0000000..e7a3d87
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/nar/StandardExtensionDiscoveringManager.java
@@ -0,0 +1,527 @@
+/*
+ * 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.AccessPolicyProvider;
+import org.apache.nifi.authorization.Authorizer;
+import org.apache.nifi.authorization.UserGroupProvider;
+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 StandardExtensionDiscoveringManager implements ExtensionDiscoveringManager {
+
+    private static final Logger logger = LoggerFactory.getLogger(StandardExtensionDiscoveringManager.class);
+
+    // Maps a service definition (interface) to those classes that implement the interface
+    private final Map<Class, Set<Class>> definitionMap = new HashMap<>();
+
+    private final Map<String, List<Bundle>> classNameBundleLookup = new HashMap<>();
+    private final Map<BundleCoordinate, Set<Class>> bundleCoordinateClassesLookup = new HashMap<>();
+    private final Map<BundleCoordinate, Bundle> bundleCoordinateBundleLookup = new HashMap<>();
+    private final Map<ClassLoader, Bundle> classLoaderBundleLookup = new HashMap<>();
+    private final Map<String, ConfigurableComponent> tempComponentLookup = new HashMap<>();
+
+    private final Map<String, Class<?>> requiresInstanceClassLoading = new HashMap<>();
+    private final Map<String, InstanceClassLoader> instanceClassloaderLookup = new ConcurrentHashMap<>();
+
+    public StandardExtensionDiscoveringManager() {
+        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(UserGroupProvider.class, new HashSet<>());
+        definitionMap.put(AccessPolicyProvider.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<>());
+    }
+
+    @Override
+    public Set<Bundle> getAllBundles() {
+        return classNameBundleLookup.values().stream()
+            .flatMap(List::stream)
+            .collect(Collectors.toSet());
+    }
+
+    @Override
+    public void discoverExtensions(final Bundle systemBundle, final Set<Bundle> narBundles) {
+        // 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);
+
+        discoverExtensions(narBundles);
+    }
+
+    @Override
+    public void discoverExtensions(final Set<Bundle> narBundles) {
+        // get the current context class loader
+        ClassLoader currentContextClassLoader = Thread.currentThread().getContextClassLoader();
+
+        // 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 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, bundleCoordinateClassesLookup, bundle, entry.getValue());
+                    }
+                }
+
+            }
+
+            classLoaderBundleLookup.put(bundle.getClassLoader(), bundle);
+        }
+    }
+
+    private void initializeTempComponent(final ConfigurableComponent configurableComponent) {
+        ConfigurableComponentInitializer initializer = null;
+        try {
+            initializer = ConfigurableComponentInitializerFactory.createComponentInitializer(this, 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 void registerServiceClass(final Class<?> type,
+                                             final Map<String, List<Bundle>> classNameBundleMap,
+                                             final Map<BundleCoordinate, Set<Class>> bundleCoordinateClassesMap,
+                                             final Bundle bundle, final Set<Class> classes) {
+        final String className = type.getName();
+        final BundleCoordinate bundleCoordinate = bundle.getBundleDetails().getCoordinate();
+
+        // get the bundles that have already been registered for the class name
+        final List<Bundle> registeredBundles = classNameBundleMap.computeIfAbsent(className, (key) -> new ArrayList<>());
+        final Set<Class> bundleCoordinateClasses = bundleCoordinateClassesMap.computeIfAbsent(bundleCoordinate, (key) -> new HashSet<>());
+
+        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(bundleCoordinate)) {
+                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);
+            bundleCoordinateClasses.add(type);
+            classes.add(type);
+
+            if (type.isAnnotationPresent(RequiresInstanceClassLoading.class)) {
+                final String cacheKey = getClassBundleKey(className, bundleCoordinate);
+                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);
+    }
+
+    @Override
+    public 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 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;
+    }
+
+    @Override
+    public InstanceClassLoader getInstanceClassLoader(final String instanceIdentifier) {
+        return instanceClassloaderLookup.get(instanceIdentifier);
+    }
+
+    @Override
+    public InstanceClassLoader removeInstanceClassLoader(final String instanceIdentifier) {
+        if (instanceIdentifier == null) {
+            return null;
+        }
+
+        final InstanceClassLoader classLoader = instanceClassloaderLookup.remove(instanceIdentifier);
+        closeURLClassLoader(instanceIdentifier, classLoader);
+        return classLoader;
+    }
+
+    @Override
+    public 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);
+            }
+        }
+    }
+
+    @Override
+    public 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);
+    }
+
+    @Override
+    public Bundle getBundle(final BundleCoordinate bundleCoordinate) {
+        if (bundleCoordinate == null) {
+            throw new IllegalArgumentException("BundleCoordinate cannot be null");
+        }
+        return bundleCoordinateBundleLookup.get(bundleCoordinate);
+    }
+
+    @Override
+    public Set<Class> getTypes(final BundleCoordinate bundleCoordinate) {
+        if (bundleCoordinate == null) {
+            throw new IllegalArgumentException("BundleCoordinate cannot be null");
+        }
+        final Set<Class> types = bundleCoordinateClassesLookup.get(bundleCoordinate);
+        return types == null ? Collections.emptySet() : Collections.unmodifiableSet(types);
+    }
+
+    @Override
+    public Bundle getBundle(final ClassLoader classLoader) {
+        if (classLoader == null) {
+            throw new IllegalArgumentException("ClassLoader cannot be null");
+        }
+        return classLoaderBundleLookup.get(classLoader);
+    }
+
+    @Override
+    public 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;
+    }
+
+    @Override
+    public 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();
+    }
+
+    @Override
+    public 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/fdd8cdbb/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
index 39014bc..0039214 100644
--- 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
@@ -16,13 +16,6 @@
  */
 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;
@@ -31,15 +24,23 @@ import org.apache.nifi.processor.exception.ProcessException;
 import org.apache.nifi.util.NiFiProperties;
 import org.junit.Test;
 
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+
 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());
+        ExtensionDiscoveringManager extensionManager = new StandardExtensionDiscoveringManager();
+        extensionManager.discoverExtensions(systemBundle, Collections.emptySet());
 
-        Object obj = NarThreadContextClassLoader.createInstance(WithPropertiesConstructor.class.getName(),
+        Object obj = NarThreadContextClassLoader.createInstance(extensionManager, WithPropertiesConstructor.class.getName(),
                 WithPropertiesConstructor.class, properties);
         assertTrue(obj instanceof WithPropertiesConstructor);
         WithPropertiesConstructor withPropertiesConstructor = (WithPropertiesConstructor) obj;
@@ -52,16 +53,18 @@ public class NarThreadContextClassLoaderTest {
         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);
+        ExtensionDiscoveringManager extensionManager = new StandardExtensionDiscoveringManager();
+        extensionManager.discoverExtensions(systemBundle, Collections.emptySet());
+        NarThreadContextClassLoader.createInstance(extensionManager, 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(),
+        ExtensionDiscoveringManager extensionManager = new StandardExtensionDiscoveringManager();
+        extensionManager.discoverExtensions(systemBundle, Collections.emptySet());
+        assertTrue(NarThreadContextClassLoader.createInstance(extensionManager, WithDefaultConstructor.class.getName(),
                 WithDefaultConstructor.class, properties) instanceof WithDefaultConstructor);
     }