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

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

http://git-wip-us.apache.org/repos/asf/nifi/blob/556f309d/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/init/ConfigurableComponentInitializerFactory.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/init/ConfigurableComponentInitializerFactory.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/init/ConfigurableComponentInitializerFactory.java
new file mode 100644
index 0000000..f6ab922
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/init/ConfigurableComponentInitializerFactory.java
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.init;
+
+import org.apache.nifi.components.ConfigurableComponent;
+import org.apache.nifi.controller.ControllerService;
+import org.apache.nifi.processor.Processor;
+import org.apache.nifi.reporting.ReportingTask;
+
+public class ConfigurableComponentInitializerFactory {
+
+    /**
+     * Returns a ConfigurableComponentInitializer for the type of component.
+     * Currently Processor, ControllerService and ReportingTask are supported.
+     *
+     * @param componentClass the class that requires a ConfigurableComponentInitializer
+     * @return a ConfigurableComponentInitializer capable of initializing that specific type of class
+     */
+    public static ConfigurableComponentInitializer createComponentInitializer(final Class<? extends ConfigurableComponent> componentClass) {
+        if (Processor.class.isAssignableFrom(componentClass)) {
+            return new ProcessorInitializer();
+        } else if (ControllerService.class.isAssignableFrom(componentClass)) {
+            return new ControllerServiceInitializer();
+        } else if (ReportingTask.class.isAssignableFrom(componentClass)) {
+            return new ReportingTaskingInitializer();
+        }
+
+        return null;
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/556f309d/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/init/ControllerServiceInitializer.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/init/ControllerServiceInitializer.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/init/ControllerServiceInitializer.java
new file mode 100644
index 0000000..21b107f
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/init/ControllerServiceInitializer.java
@@ -0,0 +1,59 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.init;
+
+import org.apache.nifi.annotation.lifecycle.OnShutdown;
+import org.apache.nifi.components.ConfigurableComponent;
+import org.apache.nifi.controller.ControllerService;
+import org.apache.nifi.controller.ControllerServiceInitializationContext;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.mock.MockComponentLogger;
+import org.apache.nifi.mock.MockConfigurationContext;
+import org.apache.nifi.mock.MockControllerServiceInitializationContext;
+import org.apache.nifi.nar.ExtensionManager;
+import org.apache.nifi.nar.NarCloseable;
+import org.apache.nifi.reporting.InitializationException;
+
+/**
+ * Initializes a ControllerService using a MockControllerServiceInitializationContext
+ *
+ *
+ */
+public class ControllerServiceInitializer implements ConfigurableComponentInitializer {
+
+    @Override
+    public void initialize(ConfigurableComponent component) throws InitializationException {
+        ControllerService controllerService = (ControllerService) component;
+        ControllerServiceInitializationContext context = new MockControllerServiceInitializationContext();
+        try (NarCloseable narCloseable = NarCloseable.withComponentNarLoader(component.getClass(), context.getIdentifier())) {
+            controllerService.initialize(context);
+        }
+    }
+
+    @Override
+    public void teardown(ConfigurableComponent component) {
+        try (NarCloseable narCloseable = NarCloseable.withComponentNarLoader(component.getClass(), component.getIdentifier())) {
+            ControllerService controllerService = (ControllerService) component;
+
+            final ComponentLog logger = new MockComponentLogger();
+            final MockConfigurationContext context = new MockConfigurationContext();
+            ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnShutdown.class, controllerService, logger, context);
+        } finally {
+            ExtensionManager.removeInstanceClassLoader(component.getIdentifier());
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/556f309d/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/init/ProcessorInitializer.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/init/ProcessorInitializer.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/init/ProcessorInitializer.java
new file mode 100644
index 0000000..06fdead
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/init/ProcessorInitializer.java
@@ -0,0 +1,58 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.init;
+
+import org.apache.nifi.annotation.lifecycle.OnShutdown;
+import org.apache.nifi.components.ConfigurableComponent;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.mock.MockComponentLogger;
+import org.apache.nifi.mock.MockProcessContext;
+import org.apache.nifi.mock.MockProcessorInitializationContext;
+import org.apache.nifi.nar.ExtensionManager;
+import org.apache.nifi.nar.NarCloseable;
+import org.apache.nifi.processor.Processor;
+import org.apache.nifi.processor.ProcessorInitializationContext;
+
+/**
+ * Initializes a Processor using a MockProcessorInitializationContext
+ *
+ *
+ */
+public class ProcessorInitializer implements ConfigurableComponentInitializer {
+
+    @Override
+    public void initialize(ConfigurableComponent component) {
+        Processor processor = (Processor) component;
+        ProcessorInitializationContext initializationContext = new MockProcessorInitializationContext();
+        try (NarCloseable narCloseable = NarCloseable.withComponentNarLoader(component.getClass(), initializationContext.getIdentifier())) {
+            processor.initialize(initializationContext);
+        }
+    }
+
+    @Override
+    public void teardown(ConfigurableComponent component) {
+        Processor processor = (Processor) component;
+        try (NarCloseable narCloseable = NarCloseable.withComponentNarLoader(component.getClass(), component.getIdentifier())) {
+
+            final ComponentLog logger = new MockComponentLogger();
+            final MockProcessContext context = new MockProcessContext();
+            ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnShutdown.class, processor, logger, context);
+        } finally {
+            ExtensionManager.removeInstanceClassLoader(component.getIdentifier());
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/556f309d/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/init/ReflectionUtils.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/init/ReflectionUtils.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/init/ReflectionUtils.java
new file mode 100644
index 0000000..22420bd
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/init/ReflectionUtils.java
@@ -0,0 +1,133 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.init;
+
+import org.apache.nifi.logging.ComponentLog;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.lang.annotation.Annotation;
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
+
+/**
+ * This class is a copy of org.apache.nifi.util.ReflectionUtils. Ultimately the
+ * documentation generation component should be moved to a place where it can
+ * depend on this directly instead of copying it in.
+ *
+ *
+ */
+public class ReflectionUtils {
+
+    private final static Logger LOG = LoggerFactory.getLogger(ReflectionUtils.class);
+
+    /**
+     * Invokes all methods on the given instance that have been annotated with
+     * the given annotation. If the signature of the method that is defined in
+     * <code>instance</code> uses 1 or more parameters, those parameters must be
+     * specified by the <code>args</code> parameter. However, if more arguments
+     * are supplied by the <code>args</code> parameter than needed, the extra
+     * arguments will be ignored.
+     *
+     * @param annotation annotation
+     * @param instance instance
+     * @param logger the ComponentLog to use for logging any errors. If null,
+     * will use own logger, but that will not generate bulletins or easily tie
+     * to the Processor's log messages.
+     * @param args args
+     * @return <code>true</code> if all appropriate methods were invoked and
+     * returned without throwing an Exception, <code>false</code> if one of the
+     * methods threw an Exception or could not be invoked; if <code>false</code>
+     * is returned, an error will have been logged.
+     */
+    public static boolean quietlyInvokeMethodsWithAnnotation(
+            final Class<? extends Annotation> annotation, final Object instance, final ComponentLog logger, final Object... args) {
+
+        for (final Method method : instance.getClass().getMethods()) {
+            if (method.isAnnotationPresent(annotation)) {
+
+                final boolean isAccessible = method.isAccessible();
+                method.setAccessible(true);
+
+                try {
+                    final Class<?>[] argumentTypes = method.getParameterTypes();
+                    if (argumentTypes.length > args.length) {
+                        if (logger == null) {
+                            LOG.error("Unable to invoke method {} on {} because method expects {} parameters but only {} were given",
+                                    new Object[]{method.getName(), instance, argumentTypes.length, args.length});
+                        } else {
+                            logger.error("Unable to invoke method {} on {} because method expects {} parameters but only {} were given",
+                                    new Object[]{method.getName(), instance, argumentTypes.length, args.length});
+                        }
+
+                        return false;
+                    }
+
+                    for (int i = 0; i < argumentTypes.length; i++) {
+                        final Class<?> argType = argumentTypes[i];
+                        if (!argType.isAssignableFrom(args[i].getClass())) {
+                            if (logger == null) {
+                                LOG.error("Unable to invoke method {} on {} because method parameter {} is expected to be of type {} but argument passed was of type {}",
+                                        new Object[]{method.getName(), instance, i, argType, args[i].getClass()});
+                            } else {
+                                logger.error("Unable to invoke method {} on {} because method parameter {} is expected to be of type {} but argument passed was of type {}",
+                                        new Object[]{method.getName(), instance, i, argType, args[i].getClass()});
+                            }
+
+                            return false;
+                        }
+                    }
+
+                    try {
+                        if (argumentTypes.length == args.length) {
+                            method.invoke(instance, args);
+                        } else {
+                            final Object[] argsToPass = new Object[argumentTypes.length];
+                            for (int i = 0; i < argsToPass.length; i++) {
+                                argsToPass[i] = args[i];
+                            }
+
+                            method.invoke(instance, argsToPass);
+                        }
+                    } catch (final InvocationTargetException ite) {
+                        if (logger == null) {
+                            LOG.error("Unable to invoke method {} on {} due to {}", new Object[]{method.getName(), instance, ite.getCause()});
+                            LOG.error("", ite.getCause());
+                        } else {
+                            logger.error("Unable to invoke method {} on {} due to {}", new Object[]{method.getName(), instance, ite.getCause()});
+                        }
+                    } catch (final IllegalAccessException | IllegalArgumentException t) {
+                        if (logger == null) {
+                            LOG.error("Unable to invoke method {} on {} due to {}", new Object[]{method.getName(), instance, t});
+                            LOG.error("", t);
+                        } else {
+                            logger.error("Unable to invoke method {} on {} due to {}", new Object[]{method.getName(), instance, t});
+                        }
+
+                        return false;
+                    }
+                } finally {
+                    if (!isAccessible) {
+                        method.setAccessible(false);
+                    }
+                }
+            }
+        }
+
+        return true;
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/556f309d/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/init/ReportingTaskingInitializer.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/init/ReportingTaskingInitializer.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/init/ReportingTaskingInitializer.java
new file mode 100644
index 0000000..f0f495d
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/init/ReportingTaskingInitializer.java
@@ -0,0 +1,57 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.init;
+
+import org.apache.nifi.annotation.lifecycle.OnShutdown;
+import org.apache.nifi.components.ConfigurableComponent;
+import org.apache.nifi.mock.MockComponentLogger;
+import org.apache.nifi.mock.MockConfigurationContext;
+import org.apache.nifi.mock.MockReportingInitializationContext;
+import org.apache.nifi.nar.ExtensionManager;
+import org.apache.nifi.nar.NarCloseable;
+import org.apache.nifi.reporting.InitializationException;
+import org.apache.nifi.reporting.ReportingInitializationContext;
+import org.apache.nifi.reporting.ReportingTask;
+
+/**
+ * Initializes a ReportingTask using a MockReportingInitializationContext;
+ *
+ *
+ */
+public class ReportingTaskingInitializer implements ConfigurableComponentInitializer {
+
+    @Override
+    public void initialize(ConfigurableComponent component) throws InitializationException {
+        ReportingTask reportingTask = (ReportingTask) component;
+        ReportingInitializationContext context = new MockReportingInitializationContext();
+        try (NarCloseable narCloseable = NarCloseable.withComponentNarLoader(component.getClass(), context.getIdentifier())) {
+            reportingTask.initialize(context);
+        }
+    }
+
+    @Override
+    public void teardown(ConfigurableComponent component) {
+        ReportingTask reportingTask = (ReportingTask) component;
+        try (NarCloseable narCloseable = NarCloseable.withComponentNarLoader(component.getClass(), component.getIdentifier())) {
+
+            final MockConfigurationContext context = new MockConfigurationContext();
+            ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnShutdown.class, reportingTask, new MockComponentLogger(), context);
+        } finally {
+            ExtensionManager.removeInstanceClassLoader(component.getIdentifier());
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/556f309d/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/mock/MockComponentLogger.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/mock/MockComponentLogger.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/mock/MockComponentLogger.java
new file mode 100644
index 0000000..920d7eb
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/mock/MockComponentLogger.java
@@ -0,0 +1,258 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.mock;
+
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.logging.LogLevel;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Stubs out the functionality of a ComponentLog so that it can
+ * be used during initialization of a component.
+ *
+ */
+public class MockComponentLogger implements ComponentLog {
+
+    private static final Logger logger = LoggerFactory
+            .getLogger(MockComponentLogger.class);
+
+    @Override
+    public void warn(String msg, Throwable t) {
+        logger.warn(msg, t);
+    }
+
+    @Override
+    public void warn(String msg, Object[] os) {
+        logger.warn(msg, os);
+    }
+
+    @Override
+    public void warn(String msg, Object[] os, Throwable t) {
+        logger.warn(msg, os);
+        logger.warn("", t);
+    }
+
+    @Override
+    public void warn(String msg) {
+        logger.warn(msg);
+    }
+
+    @Override
+    public void trace(String msg, Throwable t) {
+        logger.trace(msg, t);
+    }
+
+    @Override
+    public void trace(String msg, Object[] os) {
+        logger.trace(msg, os);
+    }
+
+    @Override
+    public void trace(String msg) {
+        logger.trace(msg);
+    }
+
+    @Override
+    public void trace(String msg, Object[] os, Throwable t) {
+        logger.trace(msg, os);
+        logger.trace("", t);
+    }
+
+    @Override
+    public boolean isWarnEnabled() {
+        return logger.isWarnEnabled();
+    }
+
+    @Override
+    public boolean isTraceEnabled() {
+        return logger.isTraceEnabled();
+    }
+
+    @Override
+    public boolean isInfoEnabled() {
+        return logger.isInfoEnabled();
+    }
+
+    @Override
+    public boolean isErrorEnabled() {
+        return logger.isErrorEnabled();
+    }
+
+    @Override
+    public boolean isDebugEnabled() {
+        return logger.isDebugEnabled();
+    }
+
+    @Override
+    public void info(String msg, Throwable t) {
+        logger.info(msg, t);
+    }
+
+    @Override
+    public void info(String msg, Object[] os) {
+        logger.info(msg, os);
+    }
+
+    @Override
+    public void info(String msg) {
+        logger.info(msg);
+
+    }
+
+    @Override
+    public void info(String msg, Object[] os, Throwable t) {
+        logger.trace(msg, os);
+        logger.trace("", t);
+
+    }
+
+    @Override
+    public String getName() {
+        return logger.getName();
+    }
+
+    @Override
+    public void error(String msg, Throwable t) {
+        logger.error(msg, t);
+    }
+
+    @Override
+    public void error(String msg, Object[] os) {
+        logger.error(msg, os);
+    }
+
+    @Override
+    public void error(String msg) {
+        logger.error(msg);
+    }
+
+    @Override
+    public void error(String msg, Object[] os, Throwable t) {
+        logger.error(msg, os);
+        logger.error("", t);
+    }
+
+    @Override
+    public void debug(String msg, Throwable t) {
+        logger.debug(msg, t);
+    }
+
+    @Override
+    public void debug(String msg, Object[] os) {
+        logger.debug(msg, os);
+    }
+
+    @Override
+    public void debug(String msg, Object[] os, Throwable t) {
+        logger.debug(msg, os);
+        logger.debug("", t);
+    }
+
+    @Override
+    public void debug(String msg) {
+        logger.debug(msg);
+    }
+
+    @Override
+    public void log(LogLevel level, String msg, Throwable t) {
+        switch (level) {
+            case DEBUG:
+                debug(msg, t);
+                break;
+            case ERROR:
+            case FATAL:
+                error(msg, t);
+                break;
+            case INFO:
+                info(msg, t);
+                break;
+            case TRACE:
+                trace(msg, t);
+                break;
+            case WARN:
+                warn(msg, t);
+                break;
+        }
+    }
+
+    @Override
+    public void log(LogLevel level, String msg, Object[] os) {
+        switch (level) {
+            case DEBUG:
+                debug(msg, os);
+                break;
+            case ERROR:
+            case FATAL:
+                error(msg, os);
+                break;
+            case INFO:
+                info(msg, os);
+                break;
+            case TRACE:
+                trace(msg, os);
+                break;
+            case WARN:
+                warn(msg, os);
+                break;
+        }
+    }
+
+    @Override
+    public void log(LogLevel level, String msg) {
+        switch (level) {
+            case DEBUG:
+                debug(msg);
+                break;
+            case ERROR:
+            case FATAL:
+                error(msg);
+                break;
+            case INFO:
+                info(msg);
+                break;
+            case TRACE:
+                trace(msg);
+                break;
+            case WARN:
+                warn(msg);
+                break;
+        }
+    }
+
+    @Override
+    public void log(LogLevel level, String msg, Object[] os, Throwable t) {
+        switch (level) {
+            case DEBUG:
+                debug(msg, os, t);
+                break;
+            case ERROR:
+            case FATAL:
+                error(msg, os, t);
+                break;
+            case INFO:
+                info(msg, os, t);
+                break;
+            case TRACE:
+                trace(msg, os, t);
+                break;
+            case WARN:
+                warn(msg, os, t);
+                break;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/556f309d/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/mock/MockConfigurationContext.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/mock/MockConfigurationContext.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/mock/MockConfigurationContext.java
new file mode 100644
index 0000000..d1e73fb
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/mock/MockConfigurationContext.java
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.mock;
+
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.PropertyValue;
+import org.apache.nifi.controller.ConfigurationContext;
+
+import java.util.Collections;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+public class MockConfigurationContext implements ConfigurationContext {
+
+    @Override
+    public PropertyValue getProperty(PropertyDescriptor property) {
+        return null;
+    }
+
+    @Override
+    public Map<PropertyDescriptor, String> getProperties() {
+        return Collections.emptyMap();
+    }
+
+    @Override
+    public String getSchedulingPeriod() {
+        return "0 secs";
+    }
+
+    @Override
+    public Long getSchedulingPeriod(final TimeUnit timeUnit) {
+        return 0L;
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/556f309d/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/mock/MockControllerServiceInitializationContext.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/mock/MockControllerServiceInitializationContext.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/mock/MockControllerServiceInitializationContext.java
new file mode 100644
index 0000000..b111ad2
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/mock/MockControllerServiceInitializationContext.java
@@ -0,0 +1,68 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.mock;
+
+import org.apache.nifi.components.state.StateManager;
+import org.apache.nifi.controller.ControllerServiceInitializationContext;
+import org.apache.nifi.controller.ControllerServiceLookup;
+import org.apache.nifi.logging.ComponentLog;
+
+import java.io.File;
+
+/**
+ * A Mock ControllerServiceInitializationContext so that ControllerServices can
+ * be initialized for the purpose of generating documentation.
+ *
+ *
+ */
+public class MockControllerServiceInitializationContext implements ControllerServiceInitializationContext {
+
+    @Override
+    public String getIdentifier() {
+        return "mock-controller-service";
+    }
+
+    @Override
+    public ControllerServiceLookup getControllerServiceLookup() {
+        return new MockControllerServiceLookup();
+    }
+
+    @Override
+    public ComponentLog getLogger() {
+        return new MockComponentLogger();
+    }
+
+    @Override
+    public StateManager getStateManager() {
+        return null;
+    }
+
+    @Override
+    public String getKerberosServicePrincipal() {
+        return null;
+    }
+
+    @Override
+    public File getKerberosServiceKeytab() {
+        return null;
+    }
+
+    @Override
+    public File getKerberosConfigurationFile() {
+        return null;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi/blob/556f309d/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/mock/MockControllerServiceLookup.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/mock/MockControllerServiceLookup.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/mock/MockControllerServiceLookup.java
new file mode 100644
index 0000000..5307ac4
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/mock/MockControllerServiceLookup.java
@@ -0,0 +1,63 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.mock;
+
+import org.apache.nifi.controller.ControllerService;
+import org.apache.nifi.controller.ControllerServiceLookup;
+
+import java.util.Collections;
+import java.util.Set;
+
+/**
+ * A Mock ControllerServiceLookup that can be used so that
+ * ConfigurableComponents can be initialized for the purpose of generating
+ * documentation
+ *
+ *
+ */
+public class MockControllerServiceLookup implements ControllerServiceLookup {
+
+    @Override
+    public ControllerService getControllerService(final String serviceIdentifier) {
+        return null;
+    }
+
+    @Override
+    public boolean isControllerServiceEnabled(final String serviceIdentifier) {
+        return false;
+    }
+
+    @Override
+    public boolean isControllerServiceEnabled(final ControllerService service) {
+        return false;
+    }
+
+    @Override
+    public Set<String> getControllerServiceIdentifiers(final Class<? extends ControllerService> serviceType) throws IllegalArgumentException {
+        return Collections.emptySet();
+    }
+
+    @Override
+    public boolean isControllerServiceEnabling(final String serviceIdentifier) {
+        return false;
+    }
+
+    @Override
+    public String getControllerServiceName(final String serviceIdentifier) {
+        return serviceIdentifier;
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/556f309d/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/mock/MockNodeTypeProvider.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/mock/MockNodeTypeProvider.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/mock/MockNodeTypeProvider.java
new file mode 100644
index 0000000..61390e1
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/mock/MockNodeTypeProvider.java
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.mock;
+
+import org.apache.nifi.controller.NodeTypeProvider;
+
+/**
+ * A Mock NodeTypeProvider that can be used so that
+ * ConfigurableComponents can be initialized for the purpose of generating
+ * documentation
+ *
+ *
+ */
+public class MockNodeTypeProvider implements NodeTypeProvider {
+
+    @Override
+    public boolean isClustered() {
+        return false;
+    }
+
+    @Override
+    public boolean isPrimary() {
+        return false;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/556f309d/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/mock/MockProcessContext.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/mock/MockProcessContext.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/mock/MockProcessContext.java
new file mode 100644
index 0000000..cf2e2cf
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/mock/MockProcessContext.java
@@ -0,0 +1,116 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.mock;
+
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.PropertyValue;
+import org.apache.nifi.components.state.StateManager;
+import org.apache.nifi.controller.ControllerServiceLookup;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.Relationship;
+
+import java.util.Collections;
+import java.util.Map;
+import java.util.Set;
+
+public class MockProcessContext implements ProcessContext {
+
+    @Override
+    public PropertyValue getProperty(PropertyDescriptor descriptor) {
+        return null;
+    }
+
+    @Override
+    public PropertyValue getProperty(String propertyName) {
+        return null;
+    }
+
+    @Override
+    public PropertyValue newPropertyValue(String rawValue) {
+        return null;
+    }
+
+    @Override
+    public void yield() {
+
+    }
+
+    @Override
+    public int getMaxConcurrentTasks() {
+        return 0;
+    }
+
+    @Override
+    public String getAnnotationData() {
+        return "";
+    }
+
+    @Override
+    public Map<PropertyDescriptor, String> getProperties() {
+        return Collections.emptyMap();
+    }
+
+    @Override
+    public String encrypt(String unencrypted) {
+        return unencrypted;
+    }
+
+    @Override
+    public String decrypt(String encrypted) {
+        return encrypted;
+    }
+
+    @Override
+    public ControllerServiceLookup getControllerServiceLookup() {
+        return new MockControllerServiceLookup();
+    }
+
+    @Override
+    public Set<Relationship> getAvailableRelationships() {
+        return Collections.emptySet();
+    }
+
+    @Override
+    public boolean hasIncomingConnection() {
+        return true;
+    }
+
+    @Override
+    public boolean hasNonLoopConnection() {
+        return true;
+    }
+
+    @Override
+    public boolean hasConnection(Relationship relationship) {
+        return false;
+    }
+
+    @Override
+    public boolean isExpressionLanguagePresent(PropertyDescriptor property) {
+        return false;
+    }
+
+    @Override
+    public StateManager getStateManager() {
+        return null;
+    }
+
+    @Override
+    public String getName() {
+        return null;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi/blob/556f309d/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/mock/MockProcessorInitializationContext.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/mock/MockProcessorInitializationContext.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/mock/MockProcessorInitializationContext.java
new file mode 100644
index 0000000..d9320b2
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/mock/MockProcessorInitializationContext.java
@@ -0,0 +1,68 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.mock;
+
+import org.apache.nifi.controller.ControllerServiceLookup;
+import org.apache.nifi.controller.NodeTypeProvider;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.processor.ProcessorInitializationContext;
+
+import java.io.File;
+
+/**
+ * A Mock ProcessorInitializationContext that can be used so that Processors can
+ * be initialized for the purpose of generating documentation.
+ *
+ *
+ */
+public class MockProcessorInitializationContext implements ProcessorInitializationContext {
+
+    @Override
+    public String getIdentifier() {
+        return "mock-processor";
+    }
+
+    @Override
+    public ComponentLog getLogger() {
+        return new MockComponentLogger();
+    }
+
+    @Override
+    public ControllerServiceLookup getControllerServiceLookup() {
+        return new MockControllerServiceLookup();
+    }
+
+    @Override
+    public NodeTypeProvider getNodeTypeProvider() {
+        return new MockNodeTypeProvider();
+    }
+
+    @Override
+    public String getKerberosServicePrincipal() {
+        return null;
+    }
+
+    @Override
+    public File getKerberosServiceKeytab() {
+        return null;
+    }
+
+    @Override
+    public File getKerberosConfigurationFile() {
+        return null;
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/556f309d/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/mock/MockReportingInitializationContext.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/mock/MockReportingInitializationContext.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/mock/MockReportingInitializationContext.java
new file mode 100644
index 0000000..630c657
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/mock/MockReportingInitializationContext.java
@@ -0,0 +1,83 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.mock;
+
+import org.apache.nifi.controller.ControllerServiceLookup;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.reporting.ReportingInitializationContext;
+import org.apache.nifi.scheduling.SchedulingStrategy;
+
+import java.io.File;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * A Mock ReportingInitializationContext that can be used to initialize a
+ * ReportingTask for the purposes of documentation generation.
+ *
+ */
+public class MockReportingInitializationContext implements ReportingInitializationContext {
+
+    @Override
+    public String getIdentifier() {
+        return "mock-reporting-task";
+    }
+
+    @Override
+    public String getName() {
+        return "";
+    }
+
+    @Override
+    public long getSchedulingPeriod(TimeUnit timeUnit) {
+        return 0;
+    }
+
+    @Override
+    public ControllerServiceLookup getControllerServiceLookup() {
+        return new MockControllerServiceLookup();
+    }
+
+    @Override
+    public String getSchedulingPeriod() {
+        return "";
+    }
+
+    @Override
+    public SchedulingStrategy getSchedulingStrategy() {
+        return SchedulingStrategy.TIMER_DRIVEN;
+    }
+
+    @Override
+    public ComponentLog getLogger() {
+        return new MockComponentLogger();
+    }
+
+    @Override
+    public String getKerberosServicePrincipal() {
+        return null;
+    }
+
+    @Override
+    public File getKerberosServiceKeytab() {
+        return null;
+    }
+
+    @Override
+    public File getKerberosConfigurationFile() {
+        return null;
+    }
+}

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

http://git-wip-us.apache.org/repos/asf/nifi/blob/556f309d/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/nar/InstanceClassLoader.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/nar/InstanceClassLoader.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/nar/InstanceClassLoader.java
new file mode 100644
index 0000000..d9e23fa
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/nar/InstanceClassLoader.java
@@ -0,0 +1,89 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.nar;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.net.URL;
+import java.net.URLClassLoader;
+import java.util.Collections;
+import java.util.LinkedHashSet;
+import java.util.Set;
+
+/**
+ * Each processor, controller service, and reporting task will have an InstanceClassLoader.
+ *
+ * The InstanceClassLoader will either be an empty pass-through to the NARClassLoader, or will contain a
+ * copy of all the NAR's resources in the case of components that @RequireInstanceClassLoading.
+ */
+public class InstanceClassLoader extends URLClassLoader {
+
+    private static final Logger logger = LoggerFactory.getLogger(InstanceClassLoader.class);
+
+    private final String identifier;
+    private final String instanceType;
+
+    private final Set<URL> instanceUrls;
+    private final Set<URL> additionalResourceUrls;
+
+    /**
+     * @param identifier the id of the component this ClassLoader was created for
+     * @param instanceUrls the urls for the instance, will either be empty or a copy of the NARs urls
+     * @param additionalResourceUrls the urls that came from runtime properties of the component
+     * @param parent the parent ClassLoader
+     */
+    public InstanceClassLoader(final String identifier, final String type, final Set<URL> instanceUrls, final Set<URL> additionalResourceUrls, final ClassLoader parent) {
+        super(combineURLs(instanceUrls, additionalResourceUrls), parent);
+        this.identifier = identifier;
+        this.instanceType = type;
+        this.instanceUrls = Collections.unmodifiableSet(
+                instanceUrls == null ? Collections.emptySet() : new LinkedHashSet<>(instanceUrls));
+        this.additionalResourceUrls = Collections.unmodifiableSet(
+                additionalResourceUrls == null ? Collections.emptySet() : new LinkedHashSet<>(additionalResourceUrls));
+    }
+
+    private static URL[] combineURLs(final Set<URL> instanceUrls, final Set<URL> additionalResourceUrls) {
+        final Set<URL> allUrls = new LinkedHashSet<>();
+
+        if (instanceUrls != null) {
+            allUrls.addAll(instanceUrls);
+        }
+
+        if (additionalResourceUrls != null) {
+            allUrls.addAll(additionalResourceUrls);
+        }
+
+        return allUrls.toArray(new URL[allUrls.size()]);
+    }
+
+    public String getIdentifier() {
+        return identifier;
+    }
+
+    public String getInstanceType() {
+        return instanceType;
+    }
+
+    public Set<URL> getInstanceUrls() {
+        return instanceUrls;
+    }
+
+    public Set<URL> getAdditionalResourceUrls() {
+        return additionalResourceUrls;
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/556f309d/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/nar/NarCloseable.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/nar/NarCloseable.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/nar/NarCloseable.java
new file mode 100644
index 0000000..88d47ff
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/nar/NarCloseable.java
@@ -0,0 +1,112 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.nar;
+
+import java.io.Closeable;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ *
+ */
+public class NarCloseable implements Closeable {
+
+    private static final Logger logger = LoggerFactory.getLogger(NarCloseable.class);
+
+    public static NarCloseable withNarLoader() {
+        final ClassLoader current = Thread.currentThread().getContextClassLoader();
+        Thread.currentThread().setContextClassLoader(NarThreadContextClassLoader.getInstance());
+        return new NarCloseable(current);
+    }
+
+    /**
+     * Sets the current thread context class loader to the specific appropriate class loader for the given
+     * component. If the component requires per-instance class loading then the class loader will be the
+     * specific class loader for instance with the given identifier, otherwise the class loader will be
+     * the NARClassLoader.
+     *
+     * @param componentClass the component class
+     * @param componentIdentifier the identifier of the component
+     * @return NarCloseable with the current thread context classloader jailed to the Nar
+     *              or instance class loader of the component
+     */
+    public static NarCloseable withComponentNarLoader(final Class componentClass, final String componentIdentifier) {
+        final ClassLoader current = Thread.currentThread().getContextClassLoader();
+
+        ClassLoader componentClassLoader = ExtensionManager.getInstanceClassLoader(componentIdentifier);
+        if (componentClassLoader == null) {
+            componentClassLoader = componentClass.getClassLoader();
+        }
+
+        Thread.currentThread().setContextClassLoader(componentClassLoader);
+        return new NarCloseable(current);
+    }
+
+    /**
+     * Sets the current thread context class loader to the provided class loader, and returns a NarCloseable that will
+     * return the current thread context class loader to it's previous state.
+     *
+     * @param componentNarLoader the class loader to set as the current thread context class loader
+     *
+     * @return NarCloseable that will return the current thread context class loader to its previous state
+     */
+    public static NarCloseable withComponentNarLoader(final ClassLoader componentNarLoader) {
+        final ClassLoader current = Thread.currentThread().getContextClassLoader();
+        Thread.currentThread().setContextClassLoader(componentNarLoader);
+        return new NarCloseable(current);
+    }
+
+    /**
+     * Creates a Closeable object that can be used to to switch to current class
+     * loader to the framework class loader and will automatically set the
+     * ClassLoader back to the previous class loader when closed
+     *
+     * @return a NarCloseable
+     */
+    public static NarCloseable withFrameworkNar() {
+        final ClassLoader frameworkClassLoader;
+        try {
+            frameworkClassLoader = NarClassLoaders.getInstance().getFrameworkBundle().getClassLoader();
+        } catch (final Exception e) {
+            // This should never happen in a running instance, but it will occur in unit tests
+            logger.error("Unable to access Framework ClassLoader due to " + e + ". Will continue without changing ClassLoaders.");
+            if (logger.isDebugEnabled()) {
+                logger.error("", e);
+            }
+
+            return new NarCloseable(null);
+        }
+
+        final ClassLoader current = Thread.currentThread().getContextClassLoader();
+        Thread.currentThread().setContextClassLoader(frameworkClassLoader);
+        return new NarCloseable(current);
+    }
+
+    private final ClassLoader toSet;
+
+    private NarCloseable(final ClassLoader toSet) {
+        this.toSet = toSet;
+    }
+
+    @Override
+    public void close() {
+        if (toSet != null) {
+            Thread.currentThread().setContextClassLoader(toSet);
+        }
+    }
+}