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

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

http://git-wip-us.apache.org/repos/asf/nifi/blob/556f309d/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/nar/NarThreadContextClassLoader.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/nar/NarThreadContextClassLoader.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/nar/NarThreadContextClassLoader.java
new file mode 100644
index 0000000..e7faa02
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/nar/NarThreadContextClassLoader.java
@@ -0,0 +1,235 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.nar;
+
+import org.apache.nifi.authentication.LoginIdentityProvider;
+import org.apache.nifi.authorization.Authorizer;
+import org.apache.nifi.bundle.Bundle;
+import org.apache.nifi.components.Validator;
+import org.apache.nifi.components.state.StateProvider;
+import org.apache.nifi.controller.ControllerService;
+import org.apache.nifi.controller.repository.ContentRepository;
+import org.apache.nifi.controller.repository.FlowFileRepository;
+import org.apache.nifi.controller.repository.FlowFileSwapManager;
+import org.apache.nifi.controller.status.history.ComponentStatusRepository;
+import org.apache.nifi.flowfile.FlowFilePrioritizer;
+import org.apache.nifi.processor.Processor;
+import org.apache.nifi.processor.io.InputStreamCallback;
+import org.apache.nifi.processor.io.OutputStreamCallback;
+import org.apache.nifi.processor.io.StreamCallback;
+import org.apache.nifi.provenance.ProvenanceRepository;
+import org.apache.nifi.reporting.ReportingTask;
+import org.apache.nifi.util.NiFiProperties;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.lang.reflect.Constructor;
+import java.lang.reflect.InvocationTargetException;
+import java.net.URL;
+import java.net.URLClassLoader;
+import java.util.ArrayList;
+import java.util.Enumeration;
+import java.util.List;
+
+/**
+ * THREAD SAFE
+ */
+public class NarThreadContextClassLoader extends URLClassLoader {
+
+    static final ContextSecurityManager contextSecurityManager = new ContextSecurityManager();
+    private final ClassLoader forward = ClassLoader.getSystemClassLoader();
+    private static final List<Class<?>> narSpecificClasses = new ArrayList<>();
+
+    static {
+        narSpecificClasses.add(Processor.class);
+        narSpecificClasses.add(FlowFilePrioritizer.class);
+        narSpecificClasses.add(ReportingTask.class);
+        narSpecificClasses.add(Validator.class);
+        narSpecificClasses.add(InputStreamCallback.class);
+        narSpecificClasses.add(OutputStreamCallback.class);
+        narSpecificClasses.add(StreamCallback.class);
+        narSpecificClasses.add(ControllerService.class);
+        narSpecificClasses.add(Authorizer.class);
+        narSpecificClasses.add(LoginIdentityProvider.class);
+        narSpecificClasses.add(ProvenanceRepository.class);
+        narSpecificClasses.add(ComponentStatusRepository.class);
+        narSpecificClasses.add(FlowFileRepository.class);
+        narSpecificClasses.add(FlowFileSwapManager.class);
+        narSpecificClasses.add(ContentRepository.class);
+        narSpecificClasses.add(StateProvider.class);
+    }
+
+    private NarThreadContextClassLoader() {
+        super(new URL[0]);
+    }
+
+    @Override
+    public void clearAssertionStatus() {
+        lookupClassLoader().clearAssertionStatus();
+    }
+
+    @Override
+    public URL getResource(String name) {
+        return lookupClassLoader().getResource(name);
+    }
+
+    @Override
+    public InputStream getResourceAsStream(String name) {
+        return lookupClassLoader().getResourceAsStream(name);
+    }
+
+    @Override
+    public Enumeration<URL> getResources(String name) throws IOException {
+        return lookupClassLoader().getResources(name);
+    }
+
+    @Override
+    public Class<?> loadClass(String name) throws ClassNotFoundException {
+        return lookupClassLoader().loadClass(name);
+    }
+
+    @Override
+    public void setClassAssertionStatus(String className, boolean enabled) {
+        lookupClassLoader().setClassAssertionStatus(className, enabled);
+    }
+
+    @Override
+    public void setDefaultAssertionStatus(boolean enabled) {
+        lookupClassLoader().setDefaultAssertionStatus(enabled);
+    }
+
+    @Override
+    public void setPackageAssertionStatus(String packageName, boolean enabled) {
+        lookupClassLoader().setPackageAssertionStatus(packageName, enabled);
+    }
+
+    private ClassLoader lookupClassLoader() {
+        final Class<?>[] classStack = contextSecurityManager.getExecutionStack();
+
+        for (Class<?> currentClass : classStack) {
+            final Class<?> narClass = findNarClass(currentClass);
+            if (narClass != null) {
+                final ClassLoader desiredClassLoader = narClass.getClassLoader();
+
+                // When new Threads are created, the new Thread inherits the ClassLoaderContext of
+                // the caller. However, the call stack of that new Thread may not trace back to any NiFi-specific
+                // code. Therefore, the NarThreadContextClassLoader will be unable to find the appropriate NAR
+                // ClassLoader. As a result, we want to set the ContextClassLoader to the NAR ClassLoader that
+                // contains the class or resource that we are looking for.
+                // This locks the current Thread into the appropriate NAR ClassLoader Context. The framework will change
+                // the ContextClassLoader back to the NarThreadContextClassLoader as appropriate via the
+                // {@link FlowEngine.beforeExecute(Thread, Runnable)} and
+                // {@link FlowEngine.afterExecute(Thread, Runnable)} methods.
+                if (desiredClassLoader instanceof NarClassLoader) {
+                    Thread.currentThread().setContextClassLoader(desiredClassLoader);
+                }
+                return desiredClassLoader;
+            }
+        }
+        return forward;
+    }
+
+    private Class<?> findNarClass(final Class<?> cls) {
+        for (final Class<?> narClass : narSpecificClasses) {
+            if (narClass.isAssignableFrom(cls)) {
+                return cls;
+            } else if (cls.getEnclosingClass() != null) {
+                return findNarClass(cls.getEnclosingClass());
+            }
+        }
+
+        return null;
+    }
+
+    private static class SingletonHolder {
+
+        public static final NarThreadContextClassLoader instance = new NarThreadContextClassLoader();
+    }
+
+    public static NarThreadContextClassLoader getInstance() {
+        return SingletonHolder.instance;
+    }
+
+    static class ContextSecurityManager extends SecurityManager {
+
+        Class<?>[] getExecutionStack() {
+            return getClassContext();
+        }
+    }
+
+    /**
+     * Constructs an instance of the given type using either default no args
+     * constructor or a constructor which takes a NiFiProperties object
+     * (preferred).
+     *
+     * @param <T> the type to create an instance for
+     * @param implementationClassName the implementation class name
+     * @param typeDefinition the type definition
+     * @param nifiProperties the NiFiProperties instance
+     * @return constructed instance
+     * @throws InstantiationException if there is an error instantiating the class
+     * @throws IllegalAccessException if there is an error accessing the type
+     * @throws ClassNotFoundException if the class cannot be found
+     */
+    public static <T> T createInstance(final String implementationClassName, final Class<T> typeDefinition, final NiFiProperties nifiProperties)
+            throws InstantiationException, IllegalAccessException, ClassNotFoundException {
+        final ClassLoader originalClassLoader = Thread.currentThread().getContextClassLoader();
+        Thread.currentThread().setContextClassLoader(NarThreadContextClassLoader.getInstance());
+        try {
+            final List<Bundle> bundles = ExtensionManager.getBundles(implementationClassName);
+            if (bundles.size() == 0) {
+                throw new IllegalStateException(String.format("The specified implementation class '%s' is not known to this nifi.", implementationClassName));
+            }
+            if (bundles.size() > 1) {
+                throw new IllegalStateException(String.format("More than one bundle was found for the specified implementation class '%s', only one is allowed.", implementationClassName));
+            }
+
+            final Bundle bundle = bundles.get(0);
+            final ClassLoader detectedClassLoaderForType = bundle.getClassLoader();
+            final Class<?> rawClass = Class.forName(implementationClassName, true, detectedClassLoaderForType);
+
+            Thread.currentThread().setContextClassLoader(detectedClassLoaderForType);
+            final Class<?> desiredClass = rawClass.asSubclass(typeDefinition);
+            if(nifiProperties == null){
+                return typeDefinition.cast(desiredClass.newInstance());
+            }
+            Constructor<?> constructor = null;
+
+            try {
+                constructor = desiredClass.getConstructor(NiFiProperties.class);
+            } catch (NoSuchMethodException nsme) {
+                try {
+                    constructor = desiredClass.getConstructor();
+                } catch (NoSuchMethodException nsme2) {
+                    throw new IllegalStateException("Failed to find constructor which takes NiFiProperties as argument as well as the default constructor on "
+                            + desiredClass.getName(), nsme2);
+                }
+            }
+            try {
+                if (constructor.getParameterTypes().length == 0) {
+                    return typeDefinition.cast(constructor.newInstance());
+                } else {
+                    return typeDefinition.cast(constructor.newInstance(nifiProperties));
+                }
+            } catch (InvocationTargetException ite) {
+                throw new IllegalStateException("Failed to instantiate a component due to (see target exception)", ite);
+            }
+        } finally {
+            Thread.currentThread().setContextClassLoader(originalClassLoader);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/556f309d/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/test/java/org/apache/nifi/nar/NarThreadContextClassLoaderTest.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/test/java/org/apache/nifi/nar/NarThreadContextClassLoaderTest.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/test/java/org/apache/nifi/nar/NarThreadContextClassLoaderTest.java
new file mode 100644
index 0000000..39014bc
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/test/java/org/apache/nifi/nar/NarThreadContextClassLoaderTest.java
@@ -0,0 +1,99 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.nar;
+
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.assertNotNull;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.nifi.bundle.Bundle;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.util.NiFiProperties;
+import org.junit.Test;
+
+public class NarThreadContextClassLoaderTest {
+
+    @Test
+    public void validateWithPropertiesConstructor() throws Exception {
+        NiFiProperties properties = NiFiProperties.createBasicNiFiProperties("src/test/resources/nifi.properties", null);
+        Bundle systemBundle = SystemBundle.create(properties);
+        ExtensionManager.discoverExtensions(systemBundle, Collections.emptySet());
+
+        Object obj = NarThreadContextClassLoader.createInstance(WithPropertiesConstructor.class.getName(),
+                WithPropertiesConstructor.class, properties);
+        assertTrue(obj instanceof WithPropertiesConstructor);
+        WithPropertiesConstructor withPropertiesConstructor = (WithPropertiesConstructor) obj;
+        assertNotNull(withPropertiesConstructor.properties);
+    }
+
+    @Test(expected = IllegalStateException.class)
+    public void validateWithPropertiesConstructorInstantiationFailure() throws Exception {
+        Map<String, String> additionalProperties = new HashMap<>();
+        additionalProperties.put("fail", "true");
+        NiFiProperties properties = NiFiProperties.createBasicNiFiProperties("src/test/resources/nifi.properties", additionalProperties);
+        Bundle systemBundle = SystemBundle.create(properties);
+        ExtensionManager.discoverExtensions(systemBundle, Collections.emptySet());
+        NarThreadContextClassLoader.createInstance(WithPropertiesConstructor.class.getName(), WithPropertiesConstructor.class, properties);
+    }
+
+    @Test
+    public void validateWithDefaultConstructor() throws Exception {
+        NiFiProperties properties = NiFiProperties.createBasicNiFiProperties("src/test/resources/nifi.properties", null);
+        Bundle systemBundle = SystemBundle.create(properties);
+        ExtensionManager.discoverExtensions(systemBundle, Collections.emptySet());
+        assertTrue(NarThreadContextClassLoader.createInstance(WithDefaultConstructor.class.getName(),
+                WithDefaultConstructor.class, properties) instanceof WithDefaultConstructor);
+    }
+
+    public static class WithPropertiesConstructor extends AbstractProcessor {
+        private NiFiProperties properties;
+
+        public WithPropertiesConstructor() {
+
+        }
+
+        public WithPropertiesConstructor(NiFiProperties properties) {
+            if (properties.getProperty("fail") != null) {
+                throw new RuntimeException("Intentional failure");
+            }
+            this.properties = properties;
+        }
+
+        @Override
+        public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException {
+
+        }
+    }
+
+    public static class WithDefaultConstructor extends AbstractProcessor {
+        public WithDefaultConstructor() {
+
+        }
+
+        @Override
+        public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException {
+
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/556f309d/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/test/java/org/apache/nifi/nar/NarUnpackerTest.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/test/java/org/apache/nifi/nar/NarUnpackerTest.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/test/java/org/apache/nifi/nar/NarUnpackerTest.java
new file mode 100644
index 0000000..ac666ca
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/test/java/org/apache/nifi/nar/NarUnpackerTest.java
@@ -0,0 +1,183 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.nar;
+
+import org.apache.nifi.util.NiFiProperties;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.io.File;
+import java.io.IOException;
+import java.net.URISyntaxException;
+import java.nio.file.FileVisitResult;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.nio.file.SimpleFileVisitor;
+import java.nio.file.attribute.BasicFileAttributes;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+
+import static java.nio.file.StandardCopyOption.REPLACE_EXISTING;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+
+public class NarUnpackerTest {
+
+    @BeforeClass
+    public static void copyResources() throws IOException {
+
+        final Path sourcePath = Paths.get("./src/test/resources");
+        final Path targetPath = Paths.get("./target");
+
+        Files.walkFileTree(sourcePath, new SimpleFileVisitor<Path>() {
+
+            @Override
+            public FileVisitResult preVisitDirectory(Path dir, BasicFileAttributes attrs)
+                    throws IOException {
+
+                Path relativeSource = sourcePath.relativize(dir);
+                Path target = targetPath.resolve(relativeSource);
+
+                Files.createDirectories(target);
+
+                return FileVisitResult.CONTINUE;
+
+            }
+
+            @Override
+            public FileVisitResult visitFile(Path file, BasicFileAttributes attrs)
+                    throws IOException {
+
+                Path relativeSource = sourcePath.relativize(file);
+                Path target = targetPath.resolve(relativeSource);
+
+                Files.copy(file, target, REPLACE_EXISTING);
+
+                return FileVisitResult.CONTINUE;
+            }
+        });
+    }
+
+    @Test
+    public void testUnpackNars() {
+
+        NiFiProperties properties = loadSpecifiedProperties("/NarUnpacker/conf/nifi.properties", Collections.EMPTY_MAP);
+
+        assertEquals("./target/NarUnpacker/lib/",
+                properties.getProperty("nifi.nar.library.directory"));
+        assertEquals("./target/NarUnpacker/lib2/",
+                properties.getProperty("nifi.nar.library.directory.alt"));
+
+        final ExtensionMapping extensionMapping = NarUnpacker.unpackNars(properties, SystemBundle.create(properties));
+
+        assertEquals(2, extensionMapping.getAllExtensionNames().size());
+
+        assertTrue(extensionMapping.getAllExtensionNames().keySet().contains("org.apache.nifi.processors.dummy.one"));
+        assertTrue(extensionMapping.getAllExtensionNames().keySet().contains("org.apache.nifi.processors.dummy.two"));
+        final File extensionsWorkingDir = properties.getExtensionsWorkingDirectory();
+        File[] extensionFiles = extensionsWorkingDir.listFiles();
+
+        Set<String> expectedNars = new HashSet<>();
+        expectedNars.add("dummy-one.nar-unpacked");
+        expectedNars.add("dummy-two.nar-unpacked");
+        assertEquals(expectedNars.size(), extensionFiles.length);
+
+        for (File extensionFile : extensionFiles) {
+            Assert.assertTrue(expectedNars.contains(extensionFile.getName()));
+        }
+    }
+
+    @Test
+    public void testUnpackNarsFromEmptyDir() throws IOException {
+
+        final File emptyDir = new File("./target/empty/dir");
+        emptyDir.delete();
+        emptyDir.deleteOnExit();
+        assertTrue(emptyDir.mkdirs());
+
+        final Map<String, String> others = new HashMap<>();
+        others.put("nifi.nar.library.directory.alt", emptyDir.toString());
+        NiFiProperties properties = loadSpecifiedProperties("/NarUnpacker/conf/nifi.properties", others);
+
+        final ExtensionMapping extensionMapping = NarUnpacker.unpackNars(properties, SystemBundle.create(properties));
+
+        assertEquals(1, extensionMapping.getAllExtensionNames().size());
+        assertTrue(extensionMapping.getAllExtensionNames().keySet().contains("org.apache.nifi.processors.dummy.one"));
+
+        final File extensionsWorkingDir = properties.getExtensionsWorkingDirectory();
+        File[] extensionFiles = extensionsWorkingDir.listFiles();
+
+        assertEquals(1, extensionFiles.length);
+        assertEquals("dummy-one.nar-unpacked", extensionFiles[0].getName());
+    }
+
+    @Test
+    public void testUnpackNarsFromNonExistantDir() {
+
+        final File nonExistantDir = new File("./target/this/dir/should/not/exist/");
+        nonExistantDir.delete();
+        nonExistantDir.deleteOnExit();
+
+        final Map<String, String> others = new HashMap<>();
+        others.put("nifi.nar.library.directory.alt", nonExistantDir.toString());
+        NiFiProperties properties = loadSpecifiedProperties("/NarUnpacker/conf/nifi.properties", others);
+
+        final ExtensionMapping extensionMapping = NarUnpacker.unpackNars(properties, SystemBundle.create(properties));
+
+        assertTrue(extensionMapping.getAllExtensionNames().keySet().contains("org.apache.nifi.processors.dummy.one"));
+
+        assertEquals(1, extensionMapping.getAllExtensionNames().size());
+
+        final File extensionsWorkingDir = properties.getExtensionsWorkingDirectory();
+        File[] extensionFiles = extensionsWorkingDir.listFiles();
+
+        assertEquals(1, extensionFiles.length);
+        assertEquals("dummy-one.nar-unpacked", extensionFiles[0].getName());
+    }
+
+    @Test
+    public void testUnpackNarsFromNonDir() throws IOException {
+
+        final File nonDir = new File("./target/file.txt");
+        nonDir.createNewFile();
+        nonDir.deleteOnExit();
+
+        final Map<String, String> others = new HashMap<>();
+        others.put("nifi.nar.library.directory.alt", nonDir.toString());
+        NiFiProperties properties = loadSpecifiedProperties("/NarUnpacker/conf/nifi.properties", others);
+
+        final ExtensionMapping extensionMapping = NarUnpacker.unpackNars(properties, SystemBundle.create(properties));
+
+        assertNull(extensionMapping);
+    }
+
+    private NiFiProperties loadSpecifiedProperties(final String propertiesFile, final Map<String, String> others) {
+        String filePath;
+        try {
+            filePath = NarUnpackerTest.class.getResource(propertiesFile).toURI().getPath();
+        } catch (URISyntaxException ex) {
+            throw new RuntimeException("Cannot load properties file due to " + ex.getLocalizedMessage(), ex);
+        }
+        return NiFiProperties.createBasicNiFiProperties(filePath, others);
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/556f309d/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/test/resources/META-INF/services/org.apache.nifi.processor.Processor
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/test/resources/META-INF/services/org.apache.nifi.processor.Processor b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/test/resources/META-INF/services/org.apache.nifi.processor.Processor
new file mode 100644
index 0000000..9d180b6
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/test/resources/META-INF/services/org.apache.nifi.processor.Processor
@@ -0,0 +1,16 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+org.apache.nifi.nar.NarThreadContextClassLoaderTest$WithPropertiesConstructor
+org.apache.nifi.nar.NarThreadContextClassLoaderTest$WithDefaultConstructor

http://git-wip-us.apache.org/repos/asf/nifi/blob/556f309d/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/test/resources/NarUnpacker/conf/nifi.properties
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/test/resources/NarUnpacker/conf/nifi.properties b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/test/resources/NarUnpacker/conf/nifi.properties
new file mode 100644
index 0000000..3a17e0d
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/test/resources/NarUnpacker/conf/nifi.properties
@@ -0,0 +1,125 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+# Core Properties #
+nifi.flow.configuration.file=./target/flow.xml.gz
+nifi.flow.configuration.archive.dir=./target/archive/
+nifi.flowcontroller.autoResumeState=true
+nifi.flowcontroller.graceful.shutdown.period=10 sec
+nifi.flowservice.writedelay.interval=2 sec
+nifi.administrative.yield.duration=30 sec
+
+nifi.reporting.task.configuration.file=./target/reporting-tasks.xml
+nifi.controller.service.configuration.file=./target/controller-services.xml
+nifi.templates.directory=./target/templates
+nifi.ui.banner.text=UI Banner Text
+nifi.ui.autorefresh.interval=30 sec
+nifi.nar.library.directory=./target/NarUnpacker/lib/
+nifi.nar.library.directory.alt=./target/NarUnpacker/lib2/
+
+nifi.nar.working.directory=./target/work/nar/
+nifi.documentation.working.directory=./target/work/docs/components
+
+# H2 Settings
+nifi.database.directory=./target/database_repository
+nifi.h2.url.append=;LOCK_TIMEOUT=25000;WRITE_DELAY=0;AUTO_SERVER=FALSE
+
+# FlowFile Repository
+nifi.flowfile.repository.directory=./target/test-repo
+nifi.flowfile.repository.partitions=1
+nifi.flowfile.repository.checkpoint.interval=2 mins
+nifi.queue.swap.threshold=20000
+nifi.swap.storage.directory=./target/test-repo/swap
+nifi.swap.in.period=5 sec
+nifi.swap.in.threads=1
+nifi.swap.out.period=5 sec
+nifi.swap.out.threads=4
+
+# Content Repository
+nifi.content.claim.max.appendable.size=10 MB
+nifi.content.claim.max.flow.files=100
+nifi.content.repository.directory.default=./target/content_repository
+
+# Provenance Repository Properties
+nifi.provenance.repository.storage.directory=./target/provenance_repository
+nifi.provenance.repository.max.storage.time=24 hours
+nifi.provenance.repository.max.storage.size=1 GB
+nifi.provenance.repository.rollover.time=30 secs
+nifi.provenance.repository.rollover.size=100 MB
+
+# Site to Site properties
+nifi.remote.input.socket.port=9990
+nifi.remote.input.secure=true
+
+# web properties #
+nifi.web.war.directory=./target/lib
+nifi.web.http.host=
+nifi.web.http.port=8080
+nifi.web.https.host=
+nifi.web.https.port=
+nifi.web.jetty.working.directory=./target/work/jetty
+
+# security properties #
+nifi.sensitive.props.key=key
+nifi.sensitive.props.algorithm=PBEWITHMD5AND256BITAES-CBC-OPENSSL
+nifi.sensitive.props.provider=BC
+
+nifi.security.keystore=
+nifi.security.keystoreType=
+nifi.security.keystorePasswd=
+nifi.security.keyPasswd=
+nifi.security.truststore=
+nifi.security.truststoreType=
+nifi.security.truststorePasswd=
+nifi.security.needClientAuth=
+nifi.security.user.authorizer=
+
+# cluster common properties (cluster manager and nodes must have same values) #
+nifi.cluster.protocol.heartbeat.interval=5 sec
+nifi.cluster.protocol.is.secure=false
+nifi.cluster.protocol.socket.timeout=30 sec
+nifi.cluster.protocol.connection.handshake.timeout=45 sec
+# if multicast is used, then nifi.cluster.protocol.multicast.xxx properties must be configured #
+nifi.cluster.protocol.use.multicast=false
+nifi.cluster.protocol.multicast.address=
+nifi.cluster.protocol.multicast.port=
+nifi.cluster.protocol.multicast.service.broadcast.delay=500 ms
+nifi.cluster.protocol.multicast.service.locator.attempts=3
+nifi.cluster.protocol.multicast.service.locator.attempts.delay=1 sec
+
+# cluster node properties (only configure for cluster nodes) #
+nifi.cluster.is.node=false
+nifi.cluster.node.address=
+nifi.cluster.node.protocol.port=
+nifi.cluster.node.protocol.threads=2
+# if multicast is not used, nifi.cluster.node.unicast.xxx must have same values as nifi.cluster.manager.xxx #
+nifi.cluster.node.unicast.manager.address=
+nifi.cluster.node.unicast.manager.protocol.port=
+nifi.cluster.node.unicast.manager.authority.provider.port=
+
+# cluster manager properties (only configure for cluster manager) #
+nifi.cluster.is.manager=false
+nifi.cluster.manager.address=
+nifi.cluster.manager.protocol.port=
+nifi.cluster.manager.authority.provider.port=
+nifi.cluster.manager.authority.provider.threads=10
+nifi.cluster.manager.node.firewall.file=
+nifi.cluster.manager.node.event.history.size=10
+nifi.cluster.manager.node.api.connection.timeout=30 sec
+nifi.cluster.manager.node.api.read.timeout=30 sec
+nifi.cluster.manager.node.api.request.threads=10
+nifi.cluster.manager.flow.retrieval.delay=5 sec
+nifi.cluster.manager.protocol.threads=10
+nifi.cluster.manager.safemode.duration=0 sec

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

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

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

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

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

http://git-wip-us.apache.org/repos/asf/nifi/blob/556f309d/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/init/ConfigurableComponentInitializer.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/init/ConfigurableComponentInitializer.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/init/ConfigurableComponentInitializer.java
deleted file mode 100644
index 9a1149c..0000000
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/init/ConfigurableComponentInitializer.java
+++ /dev/null
@@ -1,45 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.init;
-
-import org.apache.nifi.components.ConfigurableComponent;
-import org.apache.nifi.reporting.InitializationException;
-
-/**
- * An interface for initializing and tearing down a ConfigurableComponent. It is up to the
- * implementer to call "init" so that you can call
- * ConfigurableComponent.getPropertyDescriptors()
- *
- */
-public interface ConfigurableComponentInitializer {
-
-    /**
-     * Initializes a configurable component to the point that you can call
-     * getPropertyDescriptors() on it
-     *
-     * @param component the component to initialize
-     * @throws InitializationException if the component could not be initialized
-     */
-    void initialize(ConfigurableComponent component) throws InitializationException;
-
-    /**
-     * Calls the lifecycle methods that should be called when a flow is shutdown.
-     *
-     * @param component the component to initialize
-     */
-    void teardown(ConfigurableComponent component);
-}

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

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

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

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

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

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

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

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

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

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

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