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

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

http://git-wip-us.apache.org/repos/asf/nifi/blob/d90cf846/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-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 2ad5303..d31d692 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
@@ -29,6 +29,8 @@ import org.apache.nifi.authorization.resource.Authorizable;
 import org.apache.nifi.authorization.resource.ResourceFactory;
 import org.apache.nifi.authorization.user.NiFiUser;
 import org.apache.nifi.authorization.user.NiFiUserUtils;
+import org.apache.nifi.bundle.Bundle;
+import org.apache.nifi.bundle.BundleCoordinate;
 import org.apache.nifi.cluster.protocol.NodeIdentifier;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.connectable.Connectable;
@@ -88,11 +90,13 @@ import org.apache.nifi.search.SearchContext;
 import org.apache.nifi.search.SearchResult;
 import org.apache.nifi.search.Searchable;
 import org.apache.nifi.services.FlowService;
+import org.apache.nifi.util.BundleUtils;
 import org.apache.nifi.util.FormatUtils;
 import org.apache.nifi.util.NiFiProperties;
 import org.apache.nifi.web.DownloadableContent;
 import org.apache.nifi.web.NiFiCoreException;
 import org.apache.nifi.web.ResourceNotFoundException;
+import org.apache.nifi.web.api.dto.BundleDTO;
 import org.apache.nifi.web.api.dto.DocumentedTypeDTO;
 import org.apache.nifi.web.api.dto.DtoFactory;
 import org.apache.nifi.web.api.dto.provenance.AttributeDTO;
@@ -202,8 +206,8 @@ public class ControllerFacade implements Authorizable {
      * @return processor
      * @throws ProcessorInstantiationException when unable to instantiate the processor
      */
-    public ProcessorNode createTemporaryProcessor(String type) throws ProcessorInstantiationException {
-        return flowController.createProcessor(type, UUID.randomUUID().toString(), false);
+    public ProcessorNode createTemporaryProcessor(String type, BundleDTO bundle) throws ProcessorInstantiationException {
+        return flowController.createProcessor(type, UUID.randomUUID().toString(), BundleUtils.getBundle(type, bundle), false);
     }
 
     /**
@@ -213,8 +217,8 @@ public class ControllerFacade implements Authorizable {
      * @return reporting task
      * @throws ReportingTaskInstantiationException when unable to instantiate the reporting task
      */
-    public ReportingTaskNode createTemporaryReportingTask(String type) throws ReportingTaskInstantiationException {
-        return flowController.createReportingTask(type, UUID.randomUUID().toString(), false, false);
+    public ReportingTaskNode createTemporaryReportingTask(String type, BundleDTO bundle) throws ReportingTaskInstantiationException {
+        return flowController.createReportingTask(type, UUID.randomUUID().toString(), BundleUtils.getBundle(type, bundle), false, false);
     }
 
     /**
@@ -223,8 +227,8 @@ public class ControllerFacade implements Authorizable {
      * @param type type of controller service
      * @return controller service
      */
-    public ControllerServiceNode createTemporaryControllerService(String type) {
-        return flowController.createControllerService(type, UUID.randomUUID().toString(), false);
+    public ControllerServiceNode createTemporaryControllerService(String type, BundleDTO bundle) {
+        return flowController.createControllerService(type, UUID.randomUUID().toString(), BundleUtils.getBundle(type, bundle), false);
     }
 
     /**
@@ -450,10 +454,13 @@ public class ControllerFacade implements Authorizable {
     /**
      * Gets the FlowFileProcessor types that this controller supports.
      *
+     * @param bundleGroupFilter if specified, must be member of bundle group
+     * @param bundleArtifactFilter if specified, must be member of bundle artifact
+     * @param typeFilter if specified, type must match
      * @return types
      */
-    public Set<DocumentedTypeDTO> getFlowFileProcessorTypes() {
-        return dtoFactory.fromDocumentedTypes(ExtensionManager.getExtensions(Processor.class));
+    public Set<DocumentedTypeDTO> getFlowFileProcessorTypes(final String bundleGroupFilter, final String bundleArtifactFilter, final String typeFilter) {
+        return dtoFactory.fromDocumentedTypes(ExtensionManager.getExtensions(Processor.class), bundleGroupFilter, bundleArtifactFilter, typeFilter);
     }
 
     /**
@@ -462,7 +469,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));
+        return dtoFactory.fromDocumentedTypes(ExtensionManager.getExtensions(FlowFilePrioritizer.class), null, null, null);
     }
 
     /**
@@ -472,10 +479,10 @@ public class ControllerFacade implements Authorizable {
      * @param type type
      * @return whether the specified type implements the specified serviceType
      */
-    private boolean implementsServiceType(final String serviceType, final Class type) {
+    private boolean implementsServiceType(final Class serviceType, final Class type) {
         final List<Class<?>> interfaces = ClassUtils.getAllInterfaces(type);
         for (final Class i : interfaces) {
-            if (ControllerService.class.isAssignableFrom(i) && i.getName().equals(serviceType)) {
+            if (ControllerService.class.isAssignableFrom(i) && serviceType.isAssignableFrom(i)) {
                 return true;
             }
         }
@@ -487,36 +494,62 @@ public class ControllerFacade implements Authorizable {
      * Gets the ControllerService types that this controller supports.
      *
      * @param serviceType type
+     * @param serviceBundleGroup if serviceType specified, the bundle group of the serviceType
+     * @param serviceBundleArtifact if serviceType specified, the bundle artifact of the serviceType
+     * @param serviceBundleVersion if serviceType specified, the bundle version of the serviceType
+     * @param bundleGroupFilter if specified, must be member of bundle group
+     * @param bundleArtifactFilter if specified, must be member of bundle artifact
+     * @param typeFilter if specified, type must match
      * @return the ControllerService types that this controller supports
      */
-    public Set<DocumentedTypeDTO> getControllerServiceTypes(final String serviceType) {
+    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);
 
         // identify the controller services that implement the specified serviceType if applicable
-        final Set<Class> matchingServiceImplementions;
         if (serviceType != null) {
-            matchingServiceImplementions = new HashSet<>();
+            final BundleCoordinate bundleCoordinate = new BundleCoordinate(serviceBundleGroup, serviceBundleArtifact, serviceBundleVersion);
+            final Bundle csBundle = ExtensionManager.getBundle(bundleCoordinate);
+            if (csBundle == null) {
+                throw new IllegalStateException("Unable to find bundle for coordinate " + bundleCoordinate.getCoordinate());
+            }
+
+            Class serviceClass = null;
+            final ClassLoader currentContextClassLoader = Thread.currentThread().getContextClassLoader();
+            try {
+                Thread.currentThread().setContextClassLoader(csBundle.getClassLoader());
+                serviceClass = Class.forName(serviceType, false, csBundle.getClassLoader());
+            } catch (final Exception e) {
+                Thread.currentThread().setContextClassLoader(currentContextClassLoader);
+                throw new IllegalArgumentException(String.format("Unable to load %s from bundle %s: %s", serviceType, bundleCoordinate, e), e);
+            }
+
+            final Map<Class, Bundle> matchingServiceImplementations = new HashMap<>();
 
             // check each type and remove those that aren't in the specified ancestry
-            for (final Class type : serviceImplementations) {
-                if (implementsServiceType(serviceType, type)) {
-                    matchingServiceImplementions.add(type);
+            for (final Class csClass : serviceImplementations) {
+                if (implementsServiceType(serviceClass, csClass)) {
+                    matchingServiceImplementations.put(csClass, ExtensionManager.getBundle(csClass.getClassLoader()));
                 }
             }
+
+            return dtoFactory.fromDocumentedTypes(matchingServiceImplementations, bundleGroupFilter, bundleArtifactFilter, typeFilter);
         } else {
-            matchingServiceImplementions = serviceImplementations;
+            return dtoFactory.fromDocumentedTypes(serviceImplementations, bundleGroupFilter, bundleArtifactFilter, typeFilter);
         }
-
-        return dtoFactory.fromDocumentedTypes(matchingServiceImplementions);
     }
 
     /**
      * Gets the ReportingTask types that this controller supports.
      *
+     * @param bundleGroupFilter if specified, must be member of bundle group
+     * @param bundleArtifactFilter if specified, must be member of bundle artifact
+     * @param typeFilter if specified, type must match
      * @return the ReportingTask types that this controller supports
      */
-    public Set<DocumentedTypeDTO> getReportingTaskTypes() {
-        return dtoFactory.fromDocumentedTypes(ExtensionManager.getExtensions(ReportingTask.class));
+    public Set<DocumentedTypeDTO> getReportingTaskTypes(final String bundleGroupFilter, final String bundleArtifactFilter, final String typeFilter) {
+        return dtoFactory.fromDocumentedTypes(ExtensionManager.getExtensions(ReportingTask.class), bundleGroupFilter, bundleArtifactFilter, typeFilter);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/nifi/blob/d90cf846/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/ControllerServiceDAO.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/ControllerServiceDAO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/ControllerServiceDAO.java
index b79bd76..fa92425 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/ControllerServiceDAO.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/ControllerServiceDAO.java
@@ -16,8 +16,6 @@
  */
 package org.apache.nifi.web.dao;
 
-import java.util.Set;
-
 import org.apache.nifi.components.state.Scope;
 import org.apache.nifi.components.state.StateMap;
 import org.apache.nifi.controller.ConfiguredComponent;
@@ -26,6 +24,8 @@ import org.apache.nifi.controller.service.ControllerServiceNode;
 import org.apache.nifi.controller.service.ControllerServiceState;
 import org.apache.nifi.web.api.dto.ControllerServiceDTO;
 
+import java.util.Set;
+
 public interface ControllerServiceDAO {
 
     /**
@@ -35,6 +35,13 @@ public interface ControllerServiceDAO {
     boolean hasControllerService(String controllerServiceId);
 
     /**
+     * Determines whether this controller service can be created.
+     *
+     * @param controllerServiceDTO service
+     */
+    void verifyCreate(ControllerServiceDTO controllerServiceDTO);
+
+    /**
      * Creates a controller service.
      *
      * @param controllerServiceDTO The controller service DTO

http://git-wip-us.apache.org/repos/asf/nifi/blob/d90cf846/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/ProcessorDAO.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/ProcessorDAO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/ProcessorDAO.java
index b105c55..1d88161 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/ProcessorDAO.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/ProcessorDAO.java
@@ -32,6 +32,13 @@ public interface ProcessorDAO {
     boolean hasProcessor(String id);
 
     /**
+     * Verifies the specified processor can be created.
+     *
+     * @param processorDTO processor
+     */
+    void verifyCreate(ProcessorDTO processorDTO);
+
+    /**
      * Creates a new Processor.
      *
      * @param groupId The group id where this component will be created

http://git-wip-us.apache.org/repos/asf/nifi/blob/d90cf846/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/ReportingTaskDAO.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/ReportingTaskDAO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/ReportingTaskDAO.java
index ab1a130..2078268 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/ReportingTaskDAO.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/ReportingTaskDAO.java
@@ -16,14 +16,13 @@
  */
 package org.apache.nifi.web.dao;
 
-import java.util.Set;
-
 import org.apache.nifi.components.state.Scope;
 import org.apache.nifi.components.state.StateMap;
 import org.apache.nifi.controller.ReportingTaskNode;
-
 import org.apache.nifi.web.api.dto.ReportingTaskDTO;
 
+import java.util.Set;
+
 public interface ReportingTaskDAO {
 
     /**
@@ -35,6 +34,13 @@ public interface ReportingTaskDAO {
     boolean hasReportingTask(String reportingTaskId);
 
     /**
+     * Determines whether this reporting task can be create.
+     *
+     * @param reportingTaskDTO dto
+     */
+    void verifyCreate(ReportingTaskDTO reportingTaskDTO);
+
+    /**
      * Creates a reporting task.
      *
      * @param reportingTaskDTO The reporting task DTO

http://git-wip-us.apache.org/repos/asf/nifi/blob/d90cf846/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/TemplateDAO.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/TemplateDAO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/TemplateDAO.java
index 0ffece8..03442db 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/TemplateDAO.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/TemplateDAO.java
@@ -63,11 +63,12 @@ public interface TemplateDAO {
      * @param groupId group id
      * @param originX x
      * @param originY y
-     * @param templateId template id
+     * @param templateEncodingVersion template encoding version
+     * @param snippet template snippet
      * @param idGenerationSeed the seed to use for generating UUID's. May be null.
      * @return flow snippet
      */
-    FlowSnippetDTO instantiateTemplate(String groupId, Double originX, Double originY, String templateId, String idGenerationSeed);
+    FlowSnippetDTO instantiateTemplate(String groupId, Double originX, Double originY, String templateEncodingVersion, FlowSnippetDTO snippet, String idGenerationSeed);
 
     /**
      * Gets the specified template.

http://git-wip-us.apache.org/repos/asf/nifi/blob/d90cf846/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 165e7a5..42f4fd5 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
@@ -16,9 +16,15 @@
  */
 package org.apache.nifi.web.dao.impl;
 
+import org.apache.nifi.bundle.Bundle;
+import org.apache.nifi.bundle.BundleCoordinate;
 import org.apache.nifi.controller.FlowController;
 import org.apache.nifi.groups.ProcessGroup;
+import org.apache.nifi.nar.ExtensionManager;
 import org.apache.nifi.web.ResourceNotFoundException;
+import org.apache.nifi.web.api.dto.BundleDTO;
+
+import java.util.List;
 
 public abstract class ComponentDAO {
 
@@ -66,4 +72,21 @@ public abstract class ComponentDAO {
 
         return group;
     }
+
+    protected void verifyCreate(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());
+            if (bundles.stream().filter(b -> b.getBundleDetails().getCoordinate().equals(coordinate)).count() == 0) {
+                throw new IllegalStateException(String.format("%s is not known to this NiFi instance.", coordinate.toString()));
+            }
+        } else {
+            if (bundles.isEmpty()) {
+                throw new IllegalStateException(String.format("%s is not known to this NiFi instance.", type));
+            } else if (bundles.size() > 1) {
+                throw new IllegalStateException(String.format("Multiple versions of %s exist. Please specify the desired bundle.", type));
+            }
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/d90cf846/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 169cd50..c563e49 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
@@ -16,6 +16,7 @@
  */
 package org.apache.nifi.web.dao.impl;
 
+import org.apache.nifi.bundle.BundleCoordinate;
 import org.apache.nifi.components.state.Scope;
 import org.apache.nifi.components.state.StateMap;
 import org.apache.nifi.controller.ConfiguredComponent;
@@ -27,8 +28,10 @@ import org.apache.nifi.controller.service.ControllerServiceNode;
 import org.apache.nifi.controller.service.ControllerServiceProvider;
 import org.apache.nifi.controller.service.ControllerServiceState;
 import org.apache.nifi.groups.ProcessGroup;
+import org.apache.nifi.util.BundleUtils;
 import org.apache.nifi.web.NiFiCoreException;
 import org.apache.nifi.web.ResourceNotFoundException;
+import org.apache.nifi.web.api.dto.BundleDTO;
 import org.apache.nifi.web.api.dto.ControllerServiceDTO;
 import org.apache.nifi.web.dao.ComponentStateDAO;
 import org.apache.nifi.web.dao.ControllerServiceDAO;
@@ -60,6 +63,11 @@ public class StandardControllerServiceDAO extends ComponentDAO implements Contro
     }
 
     @Override
+    public void verifyCreate(final ControllerServiceDTO controllerServiceDTO) {
+        verifyCreate(controllerServiceDTO.getType(), controllerServiceDTO.getBundle());
+    }
+
+    @Override
     public ControllerServiceNode createControllerService(final ControllerServiceDTO controllerServiceDTO) {
         // ensure the type is specified
         if (controllerServiceDTO.getType() == null) {
@@ -68,7 +76,8 @@ public class StandardControllerServiceDAO extends ComponentDAO implements Contro
 
         try {
             // create the controller service
-            final ControllerServiceNode controllerService = serviceProvider.createControllerService(controllerServiceDTO.getType(), controllerServiceDTO.getId(), true);
+            final ControllerServiceNode controllerService = serviceProvider.createControllerService(
+                    controllerServiceDTO.getType(), controllerServiceDTO.getId(), BundleUtils.getBundle(controllerServiceDTO.getType(), controllerServiceDTO.getBundle()), true);
 
             // ensure we can perform the update
             verifyUpdate(controllerService, controllerServiceDTO);
@@ -136,6 +145,9 @@ public class StandardControllerServiceDAO extends ComponentDAO implements Contro
         // perform the update
         configureControllerService(controllerService, controllerServiceDTO);
 
+        // attempt to change the underlying controller service if an updated bundle is specified
+        updateBundle(controllerService, controllerServiceDTO);
+
         // enable or disable as appropriate
         if (isNotNull(controllerServiceDTO.getState())) {
             final ControllerServiceState purposedControllerServiceState = ControllerServiceState.valueOf(controllerServiceDTO.getState());
@@ -153,6 +165,19 @@ public class StandardControllerServiceDAO extends ComponentDAO implements Contro
         return controllerService;
     }
 
+    private void updateBundle(final ControllerServiceNode controllerService, final ControllerServiceDTO controllerServiceDTO) {
+        BundleDTO bundleDTO = controllerServiceDTO.getBundle();
+        if (bundleDTO != null) {
+            final BundleCoordinate incomingCoordinate = BundleUtils.getBundle(controllerService.getCanonicalClassName(), bundleDTO);
+            try {
+                flowController.changeControllerServiceType(controllerService, controllerService.getCanonicalClassName(), incomingCoordinate);
+            } catch (ControllerServiceInstantiationException e) {
+                throw new NiFiCoreException(String.format("Unable to update controller service %s from %s to %s due to: %s",
+                        controllerServiceDTO.getId(), controllerService.getBundleCoordinate().getCoordinate(), incomingCoordinate.getCoordinate(), e.getMessage()), e);
+            }
+        }
+    }
+
     @Override
     public Set<ConfiguredComponent> updateControllerServiceReferencingComponents(
             final String controllerServiceId, final ScheduledState scheduledState, final ControllerServiceState controllerServiceState) {
@@ -242,7 +267,8 @@ public class StandardControllerServiceDAO extends ComponentDAO implements Contro
         if (isAnyNotNull(controllerServiceDTO.getName(),
                 controllerServiceDTO.getAnnotationData(),
                 controllerServiceDTO.getComments(),
-                controllerServiceDTO.getProperties())) {
+                controllerServiceDTO.getProperties(),
+                controllerServiceDTO.getBundle())) {
             modificationRequest = true;
 
             // validate the request
@@ -254,6 +280,14 @@ 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);
+            // ensure we are only changing to a bundle with the same group and id, but different version
+            controllerService.verifyCanUpdateBundle(bundleCoordinate);
+        }
+
         if (modificationRequest) {
             controllerService.verifyCanUpdate();
         }

http://git-wip-us.apache.org/repos/asf/nifi/blob/d90cf846/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 fcd1e53..9b9ab4e 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
@@ -17,6 +17,7 @@
 package org.apache.nifi.web.dao.impl;
 
 import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.bundle.BundleCoordinate;
 import org.apache.nifi.components.state.Scope;
 import org.apache.nifi.components.state.StateMap;
 import org.apache.nifi.connectable.Connection;
@@ -30,11 +31,13 @@ import org.apache.nifi.controller.exception.ValidationException;
 import org.apache.nifi.groups.ProcessGroup;
 import org.apache.nifi.logging.LogLevel;
 import org.apache.nifi.processor.Relationship;
-import org.apache.nifi.scheduling.SchedulingStrategy;
 import org.apache.nifi.scheduling.ExecutionNode;
+import org.apache.nifi.scheduling.SchedulingStrategy;
+import org.apache.nifi.util.BundleUtils;
 import org.apache.nifi.util.FormatUtils;
 import org.apache.nifi.web.NiFiCoreException;
 import org.apache.nifi.web.ResourceNotFoundException;
+import org.apache.nifi.web.api.dto.BundleDTO;
 import org.apache.nifi.web.api.dto.ProcessorConfigDTO;
 import org.apache.nifi.web.api.dto.ProcessorDTO;
 import org.apache.nifi.web.dao.ComponentStateDAO;
@@ -77,6 +80,11 @@ public class StandardProcessorDAO extends ComponentDAO implements ProcessorDAO {
     }
 
     @Override
+    public void verifyCreate(final ProcessorDTO processorDTO) {
+        verifyCreate(processorDTO.getType(), processorDTO.getBundle());
+    }
+
+    @Override
     public ProcessorNode createProcessor(final String groupId, ProcessorDTO processorDTO) {
         if (processorDTO.getParentGroupId() != null && !flowController.areGroupsSame(groupId, processorDTO.getParentGroupId())) {
             throw new IllegalArgumentException("Cannot specify a different Parent Group ID than the Group to which the Processor is being added.");
@@ -92,7 +100,7 @@ public class StandardProcessorDAO extends ComponentDAO implements ProcessorDAO {
 
         try {
             // attempt to create the processor
-            ProcessorNode processor = flowController.createProcessor(processorDTO.getType(), processorDTO.getId());
+            ProcessorNode processor = flowController.createProcessor(processorDTO.getType(), processorDTO.getId(), BundleUtils.getBundle(processorDTO.getType(), processorDTO.getBundle()));
 
             // ensure we can perform the update before we add the processor to the flow
             verifyUpdate(processor, processorDTO);
@@ -342,10 +350,18 @@ public class StandardProcessorDAO extends ComponentDAO implements ProcessorDAO {
         }
 
         boolean modificationRequest = false;
-        if (isAnyNotNull(processorDTO.getName())) {
+        if (isAnyNotNull(processorDTO.getName(), processorDTO.getBundle())) {
             modificationRequest = true;
         }
 
+        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);
+            // ensure we are only changing to a bundle with the same group and id, but different version
+            processor.verifyCanUpdateBundle(bundleCoordinate);
+        }
+
         final ProcessorConfigDTO configDTO = processorDTO.getConfig();
         if (configDTO != null) {
             if (isAnyNotNull(configDTO.getAnnotationData(),
@@ -388,6 +404,9 @@ public class StandardProcessorDAO extends ComponentDAO implements ProcessorDAO {
         // configure the processor
         configureProcessor(processor, processorDTO);
 
+        // attempt to change the underlying processor if an updated bundle is specified
+        updateBundle(processor, processorDTO);
+
         // see if an update is necessary
         if (isNotNull(processorDTO.getState())) {
             final ScheduledState purposedScheduledState = ScheduledState.valueOf(processorDTO.getState());
@@ -429,6 +448,19 @@ public class StandardProcessorDAO extends ComponentDAO implements ProcessorDAO {
         return processor;
     }
 
+    private void updateBundle(ProcessorNode processor, ProcessorDTO processorDTO) {
+        BundleDTO bundleDTO = processorDTO.getBundle();
+        if (bundleDTO != null) {
+            BundleCoordinate incomingCoordinate = BundleUtils.getBundle(processor.getCanonicalClassName(), bundleDTO);
+            try {
+                flowController.changeProcessorType(processor, processor.getCanonicalClassName(), incomingCoordinate);
+            } catch (ProcessorInstantiationException e) {
+                throw new NiFiCoreException(String.format("Unable to update processor %s from %s to %s due to: %s",
+                        processorDTO.getId(), processor.getBundleCoordinate().getCoordinate(), incomingCoordinate.getCoordinate(), e.getMessage()), e);
+            }
+        }
+    }
+
     @Override
     public void verifyDelete(String processorId) {
         ProcessorNode processor = locateProcessor(processorId);

http://git-wip-us.apache.org/repos/asf/nifi/blob/d90cf846/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 1f2b155..7a4df89 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
@@ -16,32 +16,35 @@
  */
 package org.apache.nifi.web.dao.impl;
 
-import java.text.ParseException;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.RejectedExecutionException;
-import java.util.regex.Matcher;
 import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.bundle.BundleCoordinate;
 import org.apache.nifi.components.state.Scope;
 import org.apache.nifi.components.state.StateMap;
 import org.apache.nifi.controller.ReportingTaskNode;
 import org.apache.nifi.controller.ScheduledState;
 import org.apache.nifi.controller.exception.ComponentLifeCycleException;
-
 import org.apache.nifi.controller.exception.ValidationException;
 import org.apache.nifi.controller.reporting.ReportingTaskInstantiationException;
 import org.apache.nifi.controller.reporting.ReportingTaskProvider;
 import org.apache.nifi.scheduling.SchedulingStrategy;
+import org.apache.nifi.util.BundleUtils;
 import org.apache.nifi.util.FormatUtils;
 import org.apache.nifi.web.NiFiCoreException;
 import org.apache.nifi.web.ResourceNotFoundException;
+import org.apache.nifi.web.api.dto.BundleDTO;
 import org.apache.nifi.web.api.dto.ReportingTaskDTO;
 import org.apache.nifi.web.dao.ComponentStateDAO;
 import org.apache.nifi.web.dao.ReportingTaskDAO;
 import org.quartz.CronExpression;
 
+import java.text.ParseException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.RejectedExecutionException;
+import java.util.regex.Matcher;
+
 public class StandardReportingTaskDAO extends ComponentDAO implements ReportingTaskDAO {
 
     private ReportingTaskProvider reportingTaskProvider;
@@ -60,6 +63,11 @@ public class StandardReportingTaskDAO extends ComponentDAO implements ReportingT
     }
 
     @Override
+    public void verifyCreate(final ReportingTaskDTO reportingTaskDTO) {
+        verifyCreate(reportingTaskDTO.getType(), reportingTaskDTO.getBundle());
+    }
+
+    @Override
     public ReportingTaskNode createReportingTask(final ReportingTaskDTO reportingTaskDTO) {
         // ensure the type is specified
         if (reportingTaskDTO.getType() == null) {
@@ -68,7 +76,8 @@ public class StandardReportingTaskDAO extends ComponentDAO implements ReportingT
 
         try {
             // create the reporting task
-            final ReportingTaskNode reportingTask = reportingTaskProvider.createReportingTask(reportingTaskDTO.getType(), reportingTaskDTO.getId(), true);
+            final ReportingTaskNode reportingTask = reportingTaskProvider.createReportingTask(
+                    reportingTaskDTO.getType(), reportingTaskDTO.getId(), BundleUtils.getBundle(reportingTaskDTO.getType(), reportingTaskDTO.getBundle()), true);
 
             // ensure we can perform the update
             verifyUpdate(reportingTask, reportingTaskDTO);
@@ -108,6 +117,9 @@ public class StandardReportingTaskDAO extends ComponentDAO implements ReportingT
         // perform the update
         configureReportingTask(reportingTask, reportingTaskDTO);
 
+        // attempt to change the underlying processor if an updated bundle is specified
+        updateBundle(reportingTask, reportingTaskDTO);
+
         // configure scheduled state
         // see if an update is necessary
         if (isNotNull(reportingTaskDTO.getState())) {
@@ -150,6 +162,20 @@ public class StandardReportingTaskDAO extends ComponentDAO implements ReportingT
         return reportingTask;
     }
 
+    private void updateBundle(ReportingTaskNode reportingTask, ReportingTaskDTO reportingTaskDTO) {
+        BundleDTO bundleDTO = reportingTaskDTO.getBundle();
+        if (bundleDTO != null) {
+            final BundleCoordinate incomingCoordinate = BundleUtils.getBundle(reportingTask.getCanonicalClassName(), bundleDTO);
+            try {
+                reportingTaskProvider.changeReportingTaskType(reportingTask, reportingTask.getCanonicalClassName(), incomingCoordinate);
+            } catch (ReportingTaskInstantiationException e) {
+                throw new NiFiCoreException(String.format("Unable to update reporting task %s from %s to %s due to: %s",
+                        reportingTaskDTO.getId(), reportingTask.getBundleCoordinate().getCoordinate(), incomingCoordinate.getCoordinate(), e.getMessage()), e);
+            }
+
+        }
+    }
+
     private List<String> validateProposedConfiguration(final ReportingTaskNode reportingTask, final ReportingTaskDTO reportingTaskDTO) {
         final List<String> validationErrors = new ArrayList<>();
 
@@ -242,7 +268,8 @@ public class StandardReportingTaskDAO extends ComponentDAO implements ReportingT
                 reportingTaskDTO.getSchedulingStrategy(),
                 reportingTaskDTO.getSchedulingPeriod(),
                 reportingTaskDTO.getAnnotationData(),
-                reportingTaskDTO.getProperties())) {
+                reportingTaskDTO.getProperties(),
+                reportingTaskDTO.getBundle())) {
             modificationRequest = true;
 
             // validate the request
@@ -254,6 +281,14 @@ 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);
+            // ensure we are only changing to a bundle with the same group and id, but different version
+            reportingTask.verifyCanUpdateBundle(bundleCoordinate);
+        }
+
         if (modificationRequest) {
             reportingTask.verifyCanUpdate();
         }

http://git-wip-us.apache.org/repos/asf/nifi/blob/d90cf846/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardTemplateDAO.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/StandardTemplateDAO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardTemplateDAO.java
index 868e2d0..ce7b0c3 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardTemplateDAO.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardTemplateDAO.java
@@ -96,27 +96,20 @@ public class StandardTemplateDAO extends ComponentDAO implements TemplateDAO {
     }
 
     @Override
-    public FlowSnippetDTO instantiateTemplate(String groupId, Double originX, Double originY, String templateId, String idGenerationSeed) {
-        ProcessGroup group = locateProcessGroup(flowController, groupId);
-
-        // get the template id and find the template
-        Template template = getTemplate(templateId);
+    public FlowSnippetDTO instantiateTemplate(String groupId, Double originX, Double originY, String encodingVersion,
+                                              FlowSnippetDTO requestSnippet, String idGenerationSeed) {
 
-        // ensure the template could be found
-        if (template == null) {
-            throw new ResourceNotFoundException(String.format("Unable to locate template with id '%s'.", templateId));
-        }
+        ProcessGroup group = locateProcessGroup(flowController, groupId);
 
         try {
             // copy the template which pre-processes all ids
-            TemplateDTO templateDetails = template.getDetails();
-            FlowSnippetDTO snippet = snippetUtils.copy(templateDetails.getSnippet(), group, idGenerationSeed, false);
+            FlowSnippetDTO snippet = snippetUtils.copy(requestSnippet, group, idGenerationSeed, false);
 
-            // calculate scaling factors based on the template encoding version
-            // attempt to parse the encoding version
-            final FlowEncodingVersion templateEncodingVersion = FlowEncodingVersion.parse(templateDetails.getEncodingVersion());
+            // calculate scaling factors based on the template encoding version attempt to parse the encoding version.
             // get the major version, or 0 if no version could be parsed
+            final FlowEncodingVersion templateEncodingVersion = FlowEncodingVersion.parse(encodingVersion);
             int templateEncodingMajorVersion = templateEncodingVersion != null ? templateEncodingVersion.getMajorVersion() : 0;
+
             // based on the major version < 1, use the default scaling factors.  Otherwise, don't scale (use factor of 1.0)
             double factorX = templateEncodingMajorVersion < 1 ? FlowController.DEFAULT_POSITION_SCALE_FACTOR_X : 1.0;
             double factorY = templateEncodingMajorVersion < 1 ? FlowController.DEFAULT_POSITION_SCALE_FACTOR_Y : 1.0;
@@ -124,7 +117,6 @@ public class StandardTemplateDAO extends ComponentDAO implements TemplateDAO {
             // reposition and scale the template contents
             org.apache.nifi.util.SnippetUtils.moveAndScaleSnippet(snippet, originX, originY, factorX, factorY);
 
-
             // find all the child process groups in each process group in the top level of this snippet
             final List<ProcessGroupDTO> childProcessGroups  = org.apache.nifi.util.SnippetUtils.findAllProcessGroups(snippet);
             // scale (but don't reposition) child process groups

http://git-wip-us.apache.org/repos/asf/nifi/blob/d90cf846/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/groovy/org/apache/nifi/web/dao/impl/StandardTemplateDAOSpec.groovy
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/groovy/org/apache/nifi/web/dao/impl/StandardTemplateDAOSpec.groovy b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/groovy/org/apache/nifi/web/dao/impl/StandardTemplateDAOSpec.groovy
index 206be8d..de136fc 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/groovy/org/apache/nifi/web/dao/impl/StandardTemplateDAOSpec.groovy
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/groovy/org/apache/nifi/web/dao/impl/StandardTemplateDAOSpec.groovy
@@ -18,14 +18,18 @@ package org.apache.nifi.web.dao.impl
 
 import org.apache.nifi.authorization.Authorizer
 import org.apache.nifi.controller.FlowController
-import org.apache.nifi.controller.Template
 import org.apache.nifi.controller.serialization.FlowEncodingVersion
 import org.apache.nifi.controller.service.ControllerServiceProvider
 import org.apache.nifi.groups.ProcessGroup
-import org.apache.nifi.web.api.dto.*
+import org.apache.nifi.web.api.dto.BundleDTO
+import org.apache.nifi.web.api.dto.ComponentDTO
+import org.apache.nifi.web.api.dto.DtoFactory
+import org.apache.nifi.web.api.dto.FlowSnippetDTO
+import org.apache.nifi.web.api.dto.PositionDTO
+import org.apache.nifi.web.api.dto.ProcessGroupDTO
+import org.apache.nifi.web.api.dto.ProcessorConfigDTO
+import org.apache.nifi.web.api.dto.ProcessorDTO
 import org.apache.nifi.web.util.SnippetUtils
-
-import spock.lang.Ignore;
 import spock.lang.Specification
 import spock.lang.Unroll
 
@@ -61,22 +65,12 @@ class StandardTemplateDAOSpec extends Specification {
         }.flatten()
 
         when:
-        def instantiatedTemplate = standardTemplateDAO.instantiateTemplate(rootGroupId, newOriginX, newOriginY, templateId, idGenerationSeed)
+        def instantiatedTemplate = standardTemplateDAO.instantiateTemplate(rootGroupId, newOriginX, newOriginY, encodingVersion, snippet, idGenerationSeed)
 
         then:
         flowController.getGroup(_) >> { String gId ->
             def pg = Mock ProcessGroup
             pg.identifier >> gId
-            pg.findTemplate(templateId) >> { tId ->
-                def t = Mock Template
-                t.getDetails() >> { tDetails ->
-                    def td = Mock TemplateDTO
-                    td.snippet >> snippet
-                    td.encodingVersion >> encodingVersion
-                    return td
-                }
-                return t
-            }
             pg.inputPorts >> []
             pg.outputPorts >> []
             pg.processGroups >> []
@@ -116,31 +110,38 @@ class StandardTemplateDAOSpec extends Specification {
         }
 
         where:
-        rootGroupId | oldOriginX | oldOriginY | newOriginX | newOriginY | templateId | idGenerationSeed | encodingVersion | snippet
-        'g1'        | 0.0        | 0.0        | 5.0        | 5.0        | 't1'       | 'AAAA'           | null            | new FlowSnippetDTO()
-        'g1'        | 10.0       | 10.0       | 5.0        | 5.0        | 't1'       | 'AAAA'           | '0.7'           | new FlowSnippetDTO(
-                processors: [new ProcessorDTO(id:"c81f6810-0155-1000-0000-c4af042cb1559", name: 'proc1', config: new ProcessorConfigDTO(), position: new PositionDTO(x: 10, y: 10))])
-        'g1'        | 10.0       | -10.0      | 5.0        | 5.0        | 't1'       | 'AAAA'           | null           | new FlowSnippetDTO(
-                processors: [new ProcessorDTO(id:"c81f6810-0155-1000-0001-c4af042cb1559", name: 'proc2', config: new ProcessorConfigDTO(), position: new PositionDTO(x: 10, y: 10))],
+        rootGroupId | oldOriginX | oldOriginY | newOriginX | newOriginY | idGenerationSeed | encodingVersion | snippet
+        'g1'        | 0.0        | 0.0        | 5.0        | 5.0        | 'AAAA'           | null            | new FlowSnippetDTO()
+        'g1'        | 10.0       | 10.0       | 5.0        | 5.0        | 'AAAA'           | '0.7'           | new FlowSnippetDTO(
+                processors: [new ProcessorDTO(id:"c81f6810-0155-1000-0000-c4af042cb1559", name: 'proc1', bundle: new BundleDTO("org.apache.nifi", "standard", "1.0"),
+                        config: new ProcessorConfigDTO(), position: new PositionDTO(x: 10, y: 10))])
+        'g1'        | 10.0       | -10.0      | 5.0        | 5.0        | 'AAAA'           | null           | new FlowSnippetDTO(
+                processors: [new ProcessorDTO(id:"c81f6810-0155-1000-0001-c4af042cb1559", name: 'proc2', bundle: new BundleDTO("org.apache.nifi", "standard", "1.0"),
+                        config: new ProcessorConfigDTO(), position: new PositionDTO(x: 10, y: 10))],
                 processGroups: [
                         new ProcessGroupDTO(id:"c81f6810-0a55-1000-0000-c4af042cb1559", 
                                 name: 'g2',
                                 position: new PositionDTO(x: 105, y: -10),
-                                contents: new FlowSnippetDTO(processors: [new ProcessorDTO(id:"c81f6810-0155-1000-0002-c4af042cb1559", name: 'proc3', config: new ProcessorConfigDTO(), position: new PositionDTO(x: 50, y: 60))]))])
-        'g1'        | 10.0       | -10.0      | 5.0        | 5.0        | 't1'       | 'AAAA'           | '0.7'           | new FlowSnippetDTO(
-                processors: [new ProcessorDTO(id:"c81f6810-0155-1000-0003-c4af042cb1559", name: 'proc2', config: new ProcessorConfigDTO(), position: new PositionDTO(x: 10, y: 10))],
+                                contents: new FlowSnippetDTO(processors: [new ProcessorDTO(id:"c81f6810-0155-1000-0002-c4af042cb1559", name: 'proc3', bundle: new BundleDTO("org.apache.nifi", "standard", "1.0"),
+                                        config: new ProcessorConfigDTO(), position: new PositionDTO(x: 50, y: 60))]))])
+        'g1'        | 10.0       | -10.0      | 5.0        | 5.0        | 'AAAA'           | '0.7'           | new FlowSnippetDTO(
+                processors: [new ProcessorDTO(id:"c81f6810-0155-1000-0003-c4af042cb1559", name: 'proc2', bundle: new BundleDTO("org.apache.nifi", "standard", "1.0"),
+                        config: new ProcessorConfigDTO(), position: new PositionDTO(x: 10, y: 10))],
                 processGroups: [
                         new ProcessGroupDTO(id:"c81f6810-0a55-1000-0001-c4af042cb1559",
                                 name: 'g2',
                                 position: new PositionDTO(x: 105, y: -10),
-                                contents: new FlowSnippetDTO(processors: [new ProcessorDTO(id:"c81f6810-0155-1000-0004-c4af042cb1559", name: 'proc3', config: new ProcessorConfigDTO(), position: new PositionDTO(x: 50, y: 60))]))])
-        'g1'        | 10.0       | -10.0      | 5.0        | 5.0        | 't1'       | 'AAAA'           | '1.0'           | new FlowSnippetDTO(
-                processors: [new ProcessorDTO(id:"c81f6810-0155-1000-0005-c4af042cb1559", name: 'proc2', config: new ProcessorConfigDTO(), position: new PositionDTO(x: 10, y: 10))],
+                                contents: new FlowSnippetDTO(processors: [new ProcessorDTO(id:"c81f6810-0155-1000-0004-c4af042cb1559", name: 'proc3', bundle: new BundleDTO("org.apache.nifi", "standard", "1.0"),
+                                        config: new ProcessorConfigDTO(), position: new PositionDTO(x: 50, y: 60))]))])
+        'g1'        | 10.0       | -10.0      | 5.0        | 5.0        | 'AAAA'           | '1.0'           | new FlowSnippetDTO(
+                processors: [new ProcessorDTO(id:"c81f6810-0155-1000-0005-c4af042cb1559", name: 'proc2', bundle: new BundleDTO("org.apache.nifi", "standard", "1.0"),
+                        config: new ProcessorConfigDTO(), position: new PositionDTO(x: 10, y: 10))],
                 processGroups: [
                         new ProcessGroupDTO(id:"c81f6810-0a55-1000-0003-c4af042cb1559",
                                 name: 'g2',
                                 position: new PositionDTO(x: 105, y: -10),
-                                contents: new FlowSnippetDTO(processors: [new ProcessorDTO(id:"c81f6810-0155-1000-0006-c4af042cb1559", name: 'proc3', config: new ProcessorConfigDTO(), position: new PositionDTO(x: 50, y: 60))]))])
+                                contents: new FlowSnippetDTO(processors: [new ProcessorDTO(id:"c81f6810-0155-1000-0006-c4af042cb1559", name: 'proc3', bundle: new BundleDTO("org.apache.nifi", "standard", "1.0"),
+                                        config: new ProcessorConfigDTO(), position: new PositionDTO(x: 50, y: 60))]))])
     }
 
     def PositionDTO calculateMoveAndScalePosition(position, oldOriginX, oldOriginY, newOriginX, newOriginY, factorX, factorY) {

http://git-wip-us.apache.org/repos/asf/nifi/blob/d90cf846/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 d22f418..6476ed8 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
@@ -17,6 +17,7 @@
 package org.apache.nifi.integration.accesscontrol;
 
 import com.sun.jersey.api.client.ClientResponse;
+import org.apache.nifi.bundle.Bundle;
 import org.apache.nifi.integration.NiFiWebApiTest;
 import org.apache.nifi.integration.util.NiFiTestAuthorizer;
 import org.apache.nifi.integration.util.NiFiTestServer;
@@ -63,8 +64,9 @@ public class AccessControlHelper {
         flowXmlPath = props.getProperty(NiFiProperties.FLOW_CONFIGURATION_FILE);
 
         // load extensions
+        final Bundle systemBundle = ExtensionManager.createSystemBundle(props);
         NarClassLoaders.getInstance().init(props.getFrameworkWorkingDirectory(), props.getExtensionsWorkingDirectory());
-        ExtensionManager.discoverExtensions(NarClassLoaders.getInstance().getExtensionClassLoaders());
+        ExtensionManager.discoverExtensions(systemBundle, NarClassLoaders.getInstance().getBundles());
 
         // start the server
         server = new NiFiTestServer("src/main/webapp", CONTEXT_PATH, props);

http://git-wip-us.apache.org/repos/asf/nifi/blob/d90cf846/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 ee6f566..2d345e1 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
@@ -19,6 +19,7 @@ package org.apache.nifi.integration.accesscontrol;
 import com.sun.jersey.api.client.Client;
 import com.sun.jersey.api.client.ClientResponse;
 import org.apache.commons.io.FileUtils;
+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;
@@ -70,8 +71,9 @@ public class ITAccessTokenEndpoint {
         FileUtils.deleteDirectory(props.getDatabaseRepositoryPath().toFile());
 
         // load extensions
+        final Bundle systemBundle = ExtensionManager.createSystemBundle(props);
         NarClassLoaders.getInstance().init(props.getFrameworkWorkingDirectory(), props.getExtensionsWorkingDirectory());
-        ExtensionManager.discoverExtensions(NarClassLoaders.getInstance().getExtensionClassLoaders());
+        ExtensionManager.discoverExtensions(systemBundle, NarClassLoaders.getInstance().getBundles());
 
         // start the server
         SERVER = new NiFiTestServer("src/main/webapp", CONTEXT_PATH, props);

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

http://git-wip-us.apache.org/repos/asf/nifi/blob/d90cf846/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/resources/access-control/nifi.properties
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/resources/access-control/nifi.properties b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/resources/access-control/nifi.properties
index 145352f..74e368f 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/resources/access-control/nifi.properties
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/resources/access-control/nifi.properties
@@ -14,7 +14,6 @@
 # limitations under the License.
 
 # Core Properties #
-nifi.version=nifi version 
 nifi.flow.configuration.file=target/test-classes/access-control/flow.xml.gz
 nifi.flow.configuration.archive.dir=target/archive
 nifi.flowcontroller.autoResumeState=true

http://git-wip-us.apache.org/repos/asf/nifi/blob/d90cf846/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/resources/site-to-site/nifi.properties
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/resources/site-to-site/nifi.properties b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/resources/site-to-site/nifi.properties
index 4800770..500ce70 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/resources/site-to-site/nifi.properties
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/resources/site-to-site/nifi.properties
@@ -14,7 +14,6 @@
 # limitations under the License.
 
 # Core Properties #
-nifi.version=${nifi.version}
 nifi.flow.configuration.file=${nifi.flow.configuration.file}
 nifi.flow.configuration.archive.dir=${nifi.flow.configuration.archive.dir}
 nifi.flowcontroller.autoResumeState=${nifi.flowcontroller.autoResumeState}

http://git-wip-us.apache.org/repos/asf/nifi/blob/d90cf846/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-docs/pom.xml
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-docs/pom.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-docs/pom.xml
index 78259ea..3e5379d 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-docs/pom.xml
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-docs/pom.xml
@@ -43,6 +43,11 @@
             <scope>provided</scope>
         </dependency>
         <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-framework-api</artifactId>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
             <groupId>javax.servlet</groupId>
             <artifactId>javax.servlet-api</artifactId>
         </dependency>

http://git-wip-us.apache.org/repos/asf/nifi/blob/d90cf846/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-docs/src/main/java/org/apache/nifi/web/docs/DocumentationController.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-docs/src/main/java/org/apache/nifi/web/docs/DocumentationController.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-docs/src/main/java/org/apache/nifi/web/docs/DocumentationController.java
index f000bc9..80dac13 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-docs/src/main/java/org/apache/nifi/web/docs/DocumentationController.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-docs/src/main/java/org/apache/nifi/web/docs/DocumentationController.java
@@ -62,27 +62,30 @@ public class DocumentationController extends HttpServlet {
 
         // create the processors lookup
         final Map<String, String> processors = new TreeMap<>(collator);
-        for (final String processorClass : extensionMappings.getProcessorNames()) {
+        for (final String processorClass : extensionMappings.getProcessorNames().keySet()) {
             processors.put(StringUtils.substringAfterLast(processorClass, "."), processorClass);
         }
 
         // create the controller service lookup
         final Map<String, String> controllerServices = new TreeMap<>(collator);
-        for (final String controllerServiceClass : extensionMappings.getControllerServiceNames()) {
+        for (final String controllerServiceClass : extensionMappings.getControllerServiceNames().keySet()) {
             controllerServices.put(StringUtils.substringAfterLast(controllerServiceClass, "."), controllerServiceClass);
         }
 
         // create the reporting task lookup
         final Map<String, String> reportingTasks = new TreeMap<>(collator);
-        for (final String reportingTaskClass : extensionMappings.getReportingTaskNames()) {
+        for (final String reportingTaskClass : extensionMappings.getReportingTaskNames().keySet()) {
             reportingTasks.put(StringUtils.substringAfterLast(reportingTaskClass, "."), reportingTaskClass);
         }
 
         // make the available components available to the documentation jsp
         request.setAttribute("processors", processors);
+        request.setAttribute("processorBundleLookup", extensionMappings.getProcessorNames());
         request.setAttribute("controllerServices", controllerServices);
+        request.setAttribute("controllerServiceBundleLookup", extensionMappings.getControllerServiceNames());
         request.setAttribute("reportingTasks", reportingTasks);
-        request.setAttribute("totalComponents", GENERAL_LINK_COUNT + processors.size() + controllerServices.size() + reportingTasks.size() + DEVELOPER_LINK_COUNT);
+        request.setAttribute("reportingTaskBundleLookup", extensionMappings.getReportingTaskNames());
+        request.setAttribute("totalComponents", GENERAL_LINK_COUNT + extensionMappings.size() + DEVELOPER_LINK_COUNT);
 
         // forward appropriately
         request.getRequestDispatcher("/WEB-INF/jsp/documentation.jsp").forward(request, response);

http://git-wip-us.apache.org/repos/asf/nifi/blob/d90cf846/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-docs/src/main/webapp/WEB-INF/jsp/documentation.jsp
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-docs/src/main/webapp/WEB-INF/jsp/documentation.jsp b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-docs/src/main/webapp/WEB-INF/jsp/documentation.jsp
index 3325497..c424f4f 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-docs/src/main/webapp/WEB-INF/jsp/documentation.jsp
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-docs/src/main/webapp/WEB-INF/jsp/documentation.jsp
@@ -30,7 +30,18 @@
     </head>
     <body id="documentation-body">
         <div id="banner-header" class="main-banner-header"></div>
-        <span id="initial-selection" style="display: none;"><%= request.getParameter("select") == null ? "" : org.apache.nifi.util.EscapeUtils.escapeHtml(request.getParameter("select")) %></span>
+        <span id="initial-selection-type" style="display: none;">
+            <%= request.getParameter("select") == null ? "" : org.apache.nifi.util.EscapeUtils.escapeHtml(request.getParameter("select")) %>
+        </span>
+        <span id="initial-selection-bundle-group" style="display: none;">
+            <%= request.getParameter("group") == null ? "" : org.apache.nifi.util.EscapeUtils.escapeHtml(request.getParameter("group")) %>
+        </span>
+        <span id="initial-selection-bundle-artifact" style="display: none;">
+            <%= request.getParameter("artifact") == null ? "" : org.apache.nifi.util.EscapeUtils.escapeHtml(request.getParameter("artifact")) %>
+        </span>
+        <span id="initial-selection-bundle-version" style="display: none;">
+            <%= request.getParameter("version") == null ? "" : org.apache.nifi.util.EscapeUtils.escapeHtml(request.getParameter("version")) %>
+        </span>
         <div id="documentation-header" class="documentation-header">
             <div id="component-list-toggle-link">-</div>
             <div id="header-contents">
@@ -46,11 +57,11 @@
                         <div class="header">General</div>
                         <div id="general-links" class="component-links">
                             <ul>
-                                <li class="component-item"><a class="component-link overview" href="html/overview.html" target="component-usage">Overview</a></li>
-                                <li class="component-item"><a class="component-link getting-started" href="html/getting-started.html" target="component-usage">Getting Started</a></li>
-                                <li class="component-item"><a class="component-link user-guide" href="html/user-guide.html" target="component-usage">User Guide</a></li>
-                                <li class="component-item"><a class="component-link expression-language-guide" href="html/expression-language-guide.html" target="component-usage">Expression Language Guide</a></li>
-                                <li class="component-item"><a class="component-link admin-guide" href="html/administration-guide.html" target="component-usage">Admin Guide</a></li>
+                                <li class="component-item"><a class="document-link overview" href="html/overview.html" target="component-usage">Overview</a></li>
+                                <li class="component-item"><a class="document-link getting-started" href="html/getting-started.html" target="component-usage">Getting Started</a></li>
+                                <li class="component-item"><a class="document-link user-guide" href="html/user-guide.html" target="component-usage">User Guide</a></li>
+                                <li class="component-item"><a class="document-link expression-language-guide" href="html/expression-language-guide.html" target="component-usage">Expression Language Guide</a></li>
+                                <li class="component-item"><a class="document-link admin-guide" href="html/administration-guide.html" target="component-usage">Admin Guide</a></li>
                             </ul>
                             <span class="no-matching no-components hidden">No matching guides</span>
                         </div>
@@ -62,7 +73,24 @@
                                 <c:when test="${not empty processors}">
                                     <ul>
                                     <c:forEach var="entry" items="${processors}">
-                                        <li class="component-item"><a class="component-link" href="components/${entry.value}/index.html" target="component-usage">${entry.key}</a></li>
+                                        <c:forEach var="bundleEntry" items="${processorBundleLookup[entry.value]}">
+                                            <li class="component-item">
+                                                <span class="bundle-group hidden">${bundleEntry.group}</span>
+                                                <span class="bundle-artifact hidden">${bundleEntry.id}</span>
+                                                <span class="bundle-version hidden">${bundleEntry.version}</span>
+                                                <span class="extension-class hidden">${entry.value}</span>
+                                                <a class="component-link" href="components/${bundleEntry.group}/${bundleEntry.id}/${bundleEntry.version}/${entry.value}/index.html" target="component-usage">
+                                                    <c:choose>
+                                                        <c:when test="${bundleEntry.version == 'unversioned'}">
+                                                            ${entry.key}
+                                                        </c:when>
+                                                        <c:otherwise>
+                                                            ${entry.key} ${bundleEntry.version}
+                                                        </c:otherwise>
+                                                    </c:choose>
+                                                </a>
+                                            </li>
+                                        </c:forEach>
                                     </c:forEach>
                                     </ul>
                                     <span class="no-matching no-components hidden">No matching processors</span>
@@ -80,7 +108,25 @@
                                 <c:when test="${not empty controllerServices}">
                                     <ul>
                                     <c:forEach var="entry" items="${controllerServices}">
-                                        <li class="component-item"><a class="component-link" href="components/${entry.value}/index.html" target="component-usage">${entry.key}</a></li>
+                                        <c:forEach var="bundleEntry" items="${controllerServiceBundleLookup[entry.value]}">
+                                            <li class="component-item">
+                                                <span class="bundle-group hidden">${bundleEntry.group}</span>
+                                                <span class="bundle-artifact hidden">${bundleEntry.id}</span>
+                                                <span class="bundle-version hidden">${bundleEntry.version}</span>
+                                                <span class="extension-class hidden">${entry.value}</span>
+                                                <a class="component-link"
+                                                   href="components/${bundleEntry.group}/${bundleEntry.id}/${bundleEntry.version}/${entry.value}/index.html" target="component-usage">
+                                                    <c:choose>
+                                                        <c:when test="${bundleEntry.version == 'unversioned'}">
+                                                            ${entry.key}
+                                                        </c:when>
+                                                        <c:otherwise>
+                                                            ${entry.key} ${bundleEntry.version}
+                                                        </c:otherwise>
+                                                    </c:choose>
+                                                </a>
+                                            </li>
+                                        </c:forEach>
                                     </c:forEach>
                                     </ul>
                                     <span class="no-matching no-components hidden">No matching controller services</span>
@@ -98,7 +144,24 @@
                                 <c:when test="${not empty reportingTasks}">
                                     <ul>
                                     <c:forEach var="entry" items="${reportingTasks}">
-                                        <li class="component-item"><a class="component-link" href="components/${entry.value}/index.html" target="component-usage">${entry.key}</a></li>
+                                        <c:forEach var="bundleEntry" items="${reportingTaskBundleLookup[entry.value]}">
+                                            <li class="component-item">
+                                                <span class="bundle-group hidden">${bundleEntry.group}</span>
+                                                <span class="bundle-artifact hidden">${bundleEntry.id}</span>
+                                                <span class="bundle-version hidden">${bundleEntry.version}</span>
+                                                <span class="extension-class hidden">${entry.value}</span>
+                                                <a class="component-link" href="components/${bundleEntry.group}/${bundleEntry.id}/${bundleEntry.version}/${entry.value}/index.html" target="component-usage">
+                                                    <c:choose>
+                                                        <c:when test="${bundleEntry.version == 'unversioned'}">
+                                                            ${entry.key}
+                                                        </c:when>
+                                                        <c:otherwise>
+                                                            ${entry.key} ${bundleEntry.version}
+                                                        </c:otherwise>
+                                                    </c:choose>
+                                                </a>
+                                            </li>
+                                        </c:forEach>
                                     </c:forEach>
                                     </ul>
                                     <span class="no-matching no-components hidden">No matching reporting tasks</span>
@@ -113,9 +176,9 @@
                         <div class="header">Developer</div>
                         <div id="developer-links" class="component-links">
                             <ul>
-                                <li class="component-item"><a class="component-link rest-api" href="rest-api/index.html" target="component-usage">Rest Api</a></li>
-                                <li class="component-item"><a class="component-link developer-guide" href="html/developer-guide.html" target="component-usage">Developer Guide</a></li>
-                                <li class="component-item"><a class="component-link developer-guide" href="html/nifi-in-depth.html" target="component-usage">Apache NiFi In Depth</a></li>
+                                <li class="component-item"><a class="document-link rest-api" href="rest-api/index.html" target="component-usage">Rest Api</a></li>
+                                <li class="component-item"><a class="document-link developer-guide" href="html/developer-guide.html" target="component-usage">Developer Guide</a></li>
+                                <li class="component-item"><a class="document-link developer-guide" href="html/nifi-in-depth.html" target="component-usage">Apache NiFi In Depth</a></li>
                             </ul>
                             <span class="no-matching no-components hidden">No matching developer guides</span>
                         </div>

http://git-wip-us.apache.org/repos/asf/nifi/blob/d90cf846/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-docs/src/main/webapp/js/application.js
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-docs/src/main/webapp/js/application.js b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-docs/src/main/webapp/js/application.js
index d0e3c57..e0b4558 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-docs/src/main/webapp/js/application.js
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-docs/src/main/webapp/js/application.js
@@ -47,7 +47,7 @@ $(document).ready(function () {
 
     var applyComponentFilter = function (componentContainer) {
         var matchingComponents = 0;
-        var componentLinks = $(componentContainer).find('a.component-link');
+        var componentLinks = $(componentContainer).find('a.component-link, a.document-link');
 
         if (componentLinks.length === 0) {
             return matchingComponents;
@@ -108,18 +108,26 @@ $(document).ready(function () {
         $('#displayed-components').text(matchingGeneral + matchingProcessors + matchingControllerServices + matchingReportingTasks + matchingDeveloper);
     };
 
-    var selectComponent = function (componentName) {
+    var selectComponent = function (selectedExtension, selectedBundleGroup, selectedBundleArtifact, selectedArtifactVersion) {
         var componentLinks = $('a.component-link');
 
         // consider each link
         $.each(componentLinks, function () {
             var componentLink = $(this);
-            if (componentName === componentLink.text()) {
+            var item = componentLink.closest('li.component-item');
+            var extension = item.find('span.extension-class').text();
+            var group = item.find('span.bundle-group').text();
+            var artifact = item.find('span.bundle-artifact').text();
+            var version = item.find('span.bundle-version').text();
+
+            if (extension === selectedExtension && group === selectedBundleGroup
+                && artifact === selectedBundleArtifact && version === selectedArtifactVersion) {
+
                 // remove all selected styles
                 $('li.component-item').removeClass('selected');
 
                 // select this links item
-                componentLink.closest('li.component-item').addClass('selected');
+                item.addClass('selected');
 
                 // set the header
                 $('#selected-component').text(componentLink.text());
@@ -130,6 +138,28 @@ $(document).ready(function () {
         });
     };
 
+    var selectDocument = function (documentName) {
+        var documentLinks = $('a.document-link');
+
+        // consider each link
+        $.each(documentLinks, function () {
+            var documentLink = $(this);
+            if (documentName === $.trim(documentLink.text())) {
+                // remove all selected styles
+                $('li.component-item').removeClass('selected');
+
+                // select this links item
+                documentLink.closest('li.component-item').addClass('selected');
+
+                // set the header
+                $('#selected-component').text(documentLink.text());
+
+                // stop iteration
+                return false;
+            }
+        });
+    };
+
     // get the banners if we're not in the shell
     var bannerHeaderHeight = 0;
     var bannerFooterHeight = 0;
@@ -296,7 +326,7 @@ $(document).ready(function () {
             // resize window accordingly.
             $(window).resize();
 
-            var componentName = '';
+            var bundleAndComponent = '';
             var href = $(this).contents().get(0).location.href;
 
             // see if the href ends in index.htm[l]
@@ -313,42 +343,58 @@ $(document).ready(function () {
                 }
             }
 
-            // extract the simple name
+            // remove the beginning bits
             if (href.length > 0) {
-                var indexOfLastDot = href.lastIndexOf('.');
-                if (indexOfLastDot >= 0) {
-                    var indexAfterStrToFind = indexOfLastDot + 1;
-                    if (indexAfterStrToFind < href.length) {
-                        componentName = href.substr(indexAfterStrToFind);
+                var path = 'nifi-docs/components';
+                var indexOfPath = href.indexOf(path);
+                if (indexOfPath >= 0) {
+                    var indexOfBundle = indexOfPath + path.length + 1;
+                    if (indexOfBundle < href.length) {
+                        bundleAndComponent = href.substr(indexOfBundle);
                     }
                 }
             }
 
-            // if we could figure out the name
-            if (componentName !== '') {
-                selectComponent(componentName);
+            // if we could extract the bundle coordinates
+            if (bundleAndComponent !== '') {
+                var bundleTokens = bundleAndComponent.split('/');
+                if (bundleTokens.length === 4) {
+                    selectComponent(bundleTokens[3], bundleTokens[0], bundleTokens[1], bundleTokens[2]);
+                }
             }
         });
         
         // listen for on the rest api and user guide and developer guide and admin guide and overview
-        $('a.rest-api, a.user-guide, a.developer-guide, a.admin-guide, a.overview, a.expression-language-guide, a.getting-started').on('click', function() {
-            selectComponent($(this).text());
+        $('a.document-link').on('click', function() {
+            selectDocument($(this).text());
         });
 
         // get the initial selection
-        var initialComponentLink = $('a.component-link:first');
-        var initialSelection = $('#initial-selection').text();
-        if (initialSelection !== '') {
+        var initialLink = $('a.document-link:first');
+        var initialSelectionType = $.trim($('#initial-selection-type').text());
+
+        if (initialSelectionType !== '') {
+            var initialSelectionBundleGroup = $.trim($('#initial-selection-bundle-group').text());
+            var initialSelectionBundleArtifact = $.trim($('#initial-selection-bundle-artifact').text());
+            var initialSelectionBundleVersion = $.trim($('#initial-selection-bundle-version').text());
+
             $('a.component-link').each(function () {
                 var componentLink = $(this);
-                if (componentLink.text() === initialSelection) {
-                    initialComponentLink = componentLink;
+                var item = componentLink.closest('li.component-item');
+                var extension = item.find('span.extension-class').text();
+                var group = item.find('span.bundle-group').text();
+                var artifact = item.find('span.bundle-artifact').text();
+                var version = item.find('span.bundle-version').text();
+
+                if (extension === initialSelectionType && group === initialSelectionBundleGroup
+                        && artifact === initialSelectionBundleArtifact && version === initialSelectionBundleVersion) {
+                    initialLink = componentLink;
                     return false;
                 }
             });
         }
 
         // click the first link
-        initialComponentLink[0].click();
+        initialLink[0].click();
     });
 });

http://git-wip-us.apache.org/repos/asf/nifi/blob/d90cf846/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 ee978db..5c6cc01 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
@@ -23,6 +23,7 @@ import java.lang.reflect.Field;
 import java.lang.reflect.InvocationTargetException;
 import java.lang.reflect.Method;
 import java.util.HashMap;
+import java.util.List;
 import java.util.Map;
 import javax.xml.XMLConstants;
 import javax.xml.bind.JAXBContext;
@@ -45,6 +46,7 @@ import org.apache.nifi.authentication.exception.ProviderDestructionException;
 import org.apache.nifi.authentication.generated.LoginIdentityProviders;
 import org.apache.nifi.authentication.generated.Property;
 import org.apache.nifi.authentication.generated.Provider;
+import org.apache.nifi.bundle.Bundle;
 import org.apache.nifi.nar.ExtensionManager;
 import org.apache.nifi.nar.NarCloseable;
 import org.apache.nifi.properties.AESSensitivePropertyProviderFactory;
@@ -151,11 +153,19 @@ 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 ClassLoader loginIdentityProviderClassLoader = ExtensionManager.getClassLoader(loginIdentityProviderClassName);
-        if (loginIdentityProviderClassLoader == null) {
+        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));
         }
 
+        if (loginIdentityProviderBundles.size() > 1) {
+            throw new Exception(String.format("Multiple bundles found for the specified login identity provider class '%s', only one is allowed.", loginIdentityProviderClassName));
+        }
+
+        final Bundle loginIdentityProviderBundle = loginIdentityProviderBundles.get(0);
+        final ClassLoader loginIdentityProviderClassLoader = loginIdentityProviderBundle.getClassLoader();
+
         // get the current context classloader
         final ClassLoader currentClassLoader = Thread.currentThread().getContextClassLoader();
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/d90cf846/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/pom.xml
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/pom.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/pom.xml
index 08c0d46..eabff3c 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/pom.xml
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/pom.xml
@@ -459,6 +459,7 @@
                                                 <include>${staging.dir}/js/nf/canvas/nf-port-configuration.js</include>
                                                 <include>${staging.dir}/js/nf/canvas/nf-port-details.js</include>
                                                 <include>${staging.dir}/js/nf/canvas/nf-process-group-configuration.js</include>
+                                                <include>${staging.dir}/js/nf/canvas/nf-component-version.js</include>
                                                 <include>${staging.dir}/js/nf/canvas/nf-remote-process-group-configuration.js</include>
                                                 <include>${staging.dir}/js/nf/canvas/nf-remote-process-group-details.js</include>
                                                 <include>${staging.dir}/js/nf/canvas/nf-remote-process-group-ports.js</include>

http://git-wip-us.apache.org/repos/asf/nifi/blob/d90cf846/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/resources/filters/canvas.properties
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/resources/filters/canvas.properties b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/resources/filters/canvas.properties
index b5b9e33..82e2480 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/resources/filters/canvas.properties
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/resources/filters/canvas.properties
@@ -43,6 +43,7 @@ nf.canvas.script.tags=<script type="text/javascript" src="js/nf/nf-ng-bridge.js?
 <script type="text/javascript" src="js/nf/canvas/nf-port-configuration.js?${project.version}"></script>\n\
 <script type="text/javascript" src="js/nf/canvas/nf-port-details.js?${project.version}"></script>\n\
 <script type="text/javascript" src="js/nf/canvas/nf-process-group-configuration.js?${project.version}"></script>\n\
+<script type="text/javascript" src="js/nf/canvas/nf-component-version.js?${project.version}"></script>\n\
 <script type="text/javascript" src="js/nf/canvas/nf-remote-process-group-configuration.js?${project.version}"></script>\n\
 <script type="text/javascript" src="js/nf/canvas/nf-remote-process-group-details.js?${project.version}"></script>\n\
 <script type="text/javascript" src="js/nf/canvas/nf-remote-process-group-ports.js?${project.version}"></script>\n\