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

[4/4] nifi-minifi git commit: MINIFI-255 This closes #84. Update codebase to incorporate NiFi 1.2.0 handling of NARs and required dependencies MINIFI-256 Support Site to Site attaching to an interface in config

MINIFI-255 This closes #84. Update codebase to incorporate NiFi 1.2.0 handling of NARs and required dependencies
MINIFI-256 Support Site to Site attaching to an interface in config

Signed-off-by: joewitt <jo...@apache.org>


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

Branch: refs/heads/master
Commit: b556187701f38c9d1f8232913f2ba9ae64db3956
Parents: b531350
Author: Aldrin Piri <al...@apache.org>
Authored: Tue May 9 21:14:48 2017 -0400
Committer: joewitt <jo...@apache.org>
Committed: Tue May 9 23:16:24 2017 -0400

----------------------------------------------------------------------
 NOTICE                                          |    2 +
 minifi-assembly/pom.xml                         |    4 +
 .../src/main/assembly/dependencies.xml          |    2 -
 .../bootstrap/util/ConfigTransformer.java       |    3 +-
 minifi-bootstrap/src/test/resources/config.yml  |    1 +
 .../schema/RemoteProcessGroupSchema.java        |   15 +
 .../schema/RemoteProcessGroupSchemaTest.java    |   12 +
 .../init/ConfigurableComponentInitializer.java  |   45 +
 ...ConfigurableComponentInitializerFactory.java |   44 +
 .../nifi/init/ControllerServiceInitializer.java |   59 +
 .../apache/nifi/init/ProcessorInitializer.java  |   58 +
 .../org/apache/nifi/init/ReflectionUtils.java   |  133 +++
 .../nifi/init/ReportingTaskingInitializer.java  |   57 +
 .../apache/nifi/mock/MockComponentLogger.java   |  258 +++++
 .../nifi/mock/MockConfigurationContext.java     |   48 +
 ...kControllerServiceInitializationContext.java |   68 ++
 .../nifi/mock/MockControllerServiceLookup.java  |   63 ++
 .../apache/nifi/mock/MockNodeTypeProvider.java  |   40 +
 .../apache/nifi/mock/MockProcessContext.java    |  116 ++
 .../MockProcessorInitializationContext.java     |   68 ++
 .../MockReportingInitializationContext.java     |   83 ++
 .../org/apache/nifi/nar/ExtensionManager.java   |  445 ++++++--
 .../apache/nifi/nar/InstanceClassLoader.java    |  139 +--
 .../java/org/apache/nifi/nar/NarBundleUtil.java |   74 ++
 .../org/apache/nifi/nar/NarClassLoaders.java    |  199 ++--
 .../java/org/apache/nifi/nar/NarCloseable.java  |   22 +-
 .../org/apache/nifi/nar/NarManifestEntry.java   |   48 +
 .../nifi/nar/NarThreadContextClassLoader.java   |   21 +-
 .../java/org/apache/nifi/nar/SystemBundle.java  |   57 +
 .../src/main/resources/conf/logback.xml         |    2 +-
 .../java/org/apache/nifi/minifi/MiNiFi.java     |   28 +-
 .../MiNiFiPersistentProvenanceRepository.java   | 1013 +++++++++++++-----
 ...iNiFiPersistentProvenanceRepositoryTest.java |  638 +++++++++--
 .../dto/RemoteProcessGroupSchemaFunction.java   |    1 +
 .../toolkit/configuration/ConfigMainTest.java   |    7 +-
 .../dto/RemoteProcessGroupSchemaTest.java       |   16 +
 .../resources/InvokeHttpMiNiFiTemplateTest.xml  |    1 +
 .../resources/InvokeHttpMiNiFiTemplateTest.yml  |    1 +
 .../ProcessGroupsAndRemoteProcessGroups.xml     |    2 +
 .../ProcessGroupsAndRemoteProcessGroups.yml     |    2 +
 .../test/resources/SimpleRPGToLogAttributes.yml |    1 +
 .../src/test/resources/SimpleTailFileToRPG.xml  |    1 +
 .../src/test/resources/SimpleTailFileToRPG.yml  |    1 +
 pom.xml                                         |   15 +-
 44 files changed, 3204 insertions(+), 709 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/b5561877/NOTICE
----------------------------------------------------------------------
diff --git a/NOTICE b/NOTICE
index c1edd0c..6ee6948 100644
--- a/NOTICE
+++ b/NOTICE
@@ -3,3 +3,5 @@ Copyright 2014-2017 The Apache Software Foundation
 
 This product includes software developed at
 The Apache Software Foundation (http://www.apache.org/).
+
+This includes derived works from the Apache NiFi (ASLv2) project including numerous source files from the core framework API.  

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/b5561877/minifi-assembly/pom.xml
----------------------------------------------------------------------
diff --git a/minifi-assembly/pom.xml b/minifi-assembly/pom.xml
index 1ce48a7..f28ee89 100644
--- a/minifi-assembly/pom.xml
+++ b/minifi-assembly/pom.xml
@@ -187,6 +187,10 @@ limitations under the License.
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-persistent-provenance-repository</artifactId>
         </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-distributed-cache-client-service-api</artifactId>
+        </dependency>
 
         <!-- Provided in NiFi so must include here too -->
         <dependency>

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/b5561877/minifi-assembly/src/main/assembly/dependencies.xml
----------------------------------------------------------------------
diff --git a/minifi-assembly/src/main/assembly/dependencies.xml b/minifi-assembly/src/main/assembly/dependencies.xml
index d77b098..0605f23 100644
--- a/minifi-assembly/src/main/assembly/dependencies.xml
+++ b/minifi-assembly/src/main/assembly/dependencies.xml
@@ -36,7 +36,6 @@
                 <exclude>minifi-bootstrap</exclude>
                 <exclude>minifi-resources</exclude>
                 <!-- Filter items introduced via transitive dependencies that are provided in associated NARs -->
-                <exclude>zookeeper</exclude>
                 <exclude>spring-aop</exclude>
                 <exclude>spring-context</exclude>
                 <exclude>spring-beans</exclude>
@@ -46,7 +45,6 @@
                 <exclude>jaxb-impl</exclude>
                 <exclude>mail</exclude>
                 <exclude>log4j</exclude>
-                <exclude>lucene-analyzers-common</exclude>
                 <exclude>lucene-queryparser</exclude>
                 <exclude>commons-net</exclude>
                 <exclude>spring-context</exclude>

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/b5561877/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/util/ConfigTransformer.java
----------------------------------------------------------------------
diff --git a/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/util/ConfigTransformer.java b/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/util/ConfigTransformer.java
index edd4a36..e01c256 100644
--- a/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/util/ConfigTransformer.java
+++ b/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/util/ConfigTransformer.java
@@ -77,7 +77,7 @@ import java.util.zip.GZIPOutputStream;
 
 public final class ConfigTransformer {
     // Underlying version of NIFI will be using
-    public static final String NIFI_VERSION = "1.1.0";
+    public static final String NIFI_VERSION = "1.2.0";
     public static final String ROOT_GROUP = "Root-Group";
     public static final String DEFAULT_PROV_REPORTING_TASK_CLASS = "org.apache.nifi.reporting.SiteToSiteProvenanceReportingTask";
     public static final String NIFI_VERSION_KEY = "nifi.version";
@@ -576,6 +576,7 @@ public final class ConfigTransformer {
             for (RemotePortSchema remoteOutputPortSchema : remoteOutputPorts) {
                 addRemoteGroupPort(element, remoteOutputPortSchema, "outputPort");
             }
+            addTextElement(element, "networkInterface", remoteProcessGroupProperties.getLocalNetworkInterface());
 
             parentElement.appendChild(element);
         } catch (Exception e) {

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/b5561877/minifi-bootstrap/src/test/resources/config.yml
----------------------------------------------------------------------
diff --git a/minifi-bootstrap/src/test/resources/config.yml b/minifi-bootstrap/src/test/resources/config.yml
index 3b20902..e18d74a 100644
--- a/minifi-bootstrap/src/test/resources/config.yml
+++ b/minifi-bootstrap/src/test/resources/config.yml
@@ -87,6 +87,7 @@ Remote Process Groups:
   url: https://localhost:8090/nifi
   comment: ''
   timeout: 30 secs
+  local network interface: eth1
   yield period: 10 sec
   Input Ports:
   - id: 8644cbcc-a45c-40e0-964d-5e536e2ada61

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/b5561877/minifi-commons/minifi-commons-schema/src/main/java/org/apache/nifi/minifi/commons/schema/RemoteProcessGroupSchema.java
----------------------------------------------------------------------
diff --git a/minifi-commons/minifi-commons-schema/src/main/java/org/apache/nifi/minifi/commons/schema/RemoteProcessGroupSchema.java b/minifi-commons/minifi-commons-schema/src/main/java/org/apache/nifi/minifi/commons/schema/RemoteProcessGroupSchema.java
index 5a7593a..c22deff 100644
--- a/minifi-commons/minifi-commons-schema/src/main/java/org/apache/nifi/minifi/commons/schema/RemoteProcessGroupSchema.java
+++ b/minifi-commons/minifi-commons-schema/src/main/java/org/apache/nifi/minifi/commons/schema/RemoteProcessGroupSchema.java
@@ -38,6 +38,7 @@ public class RemoteProcessGroupSchema extends BaseSchemaWithIdAndName {
     public static final String PROXY_PORT_KEY = "proxy port";
     public static final String PROXY_USER_KEY = "proxy user";
     public static final String PROXY_PASSWORD_KEY = "proxy password";
+    public static final String LOCAL_NETWORK_INTERFACE_KEY = "local network interface";
 
     public static final String EXPECTED_PROXY_HOST_IF_PROXY_PORT = "expected " + PROXY_HOST_KEY + " to be set if " + PROXY_PORT_KEY + " is";
     public static final String EXPECTED_PROXY_HOST_IF_PROXY_USER = "expected " + PROXY_HOST_KEY + " to be set if " + PROXY_USER_KEY + " is";
@@ -60,6 +61,7 @@ public class RemoteProcessGroupSchema extends BaseSchemaWithIdAndName {
     public static final Integer DEFAULT_PROXY_PORT = null;
     public static final String DEFAULT_PROXY_USER = "";
     public static final String DEFAULT_PROXY_PASSWORD = "";
+    public static final String DEFAULT_NETWORK_INTERFACE = "";
 
     private String url;
     private List<RemotePortSchema> inputPorts;
@@ -73,6 +75,7 @@ public class RemoteProcessGroupSchema extends BaseSchemaWithIdAndName {
     private Integer proxyPort = DEFAULT_PROXY_PORT;
     private String proxyUser = DEFAULT_PROXY_USER;
     private String proxyPassword = DEFAULT_PROXY_PASSWORD;
+    private String localNetworkInterface = DEFAULT_NETWORK_INTERFACE;
 
     public RemoteProcessGroupSchema(Map map) {
         super(map, "RemoteProcessGroup(id: {id}, name: {name})");
@@ -98,6 +101,8 @@ public class RemoteProcessGroupSchema extends BaseSchemaWithIdAndName {
             addValidationIssue(TRANSPORT_PROTOCOL_KEY, wrapperName, "it must be either 'RAW' or 'HTTP' but is '" + transportProtocol + "'");
         }
 
+        localNetworkInterface = getOptionalKeyAsType(map, LOCAL_NETWORK_INTERFACE_KEY, String.class, wrapperName, DEFAULT_NETWORK_INTERFACE);
+
         proxyHost = getOptionalKeyAsType(map, PROXY_HOST_KEY, String.class, wrapperName, DEFAULT_PROXY_HOST);
         proxyPort = getOptionalKeyAsType(map, PROXY_PORT_KEY, Integer.class, wrapperName, DEFAULT_PROXY_PORT);
         proxyUser = getOptionalKeyAsType(map, PROXY_USER_KEY, String.class, wrapperName, DEFAULT_PROXY_USER);
@@ -121,6 +126,7 @@ public class RemoteProcessGroupSchema extends BaseSchemaWithIdAndName {
         } else if (StringUtil.isNullOrEmpty(proxyPassword)) {
             addValidationIssue(PROXY_USER_KEY, wrapperName, EXPECTED_PROXY_PASSWORD_IF_PROXY_USER);
         }
+
     }
 
     @Override
@@ -135,6 +141,7 @@ public class RemoteProcessGroupSchema extends BaseSchemaWithIdAndName {
         result.put(PROXY_PORT_KEY, proxyPort == null ? "" : proxyPort);
         result.put(PROXY_USER_KEY, proxyUser);
         result.put(PROXY_PASSWORD_KEY, proxyPassword);
+        result.put(LOCAL_NETWORK_INTERFACE_KEY, localNetworkInterface);
         putListIfNotNull(result, INPUT_PORTS_KEY, inputPorts);
         putListIfNotNull(result, OUTPUT_PORTS_KEY, outputPorts);
         return result;
@@ -203,4 +210,12 @@ public class RemoteProcessGroupSchema extends BaseSchemaWithIdAndName {
     public String getProxyPassword() {
         return proxyPassword;
     }
+
+    public void setLocalNetworkInterface(String LocalNetworkInterface) {
+        this.localNetworkInterface = LocalNetworkInterface;
+    }
+
+    public String getLocalNetworkInterface() {
+        return localNetworkInterface;
+    }
 }

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/b5561877/minifi-commons/minifi-commons-schema/src/test/java/org/apache/nifi/minifi/commons/schema/RemoteProcessGroupSchemaTest.java
----------------------------------------------------------------------
diff --git a/minifi-commons/minifi-commons-schema/src/test/java/org/apache/nifi/minifi/commons/schema/RemoteProcessGroupSchemaTest.java b/minifi-commons/minifi-commons-schema/src/test/java/org/apache/nifi/minifi/commons/schema/RemoteProcessGroupSchemaTest.java
index 2339f40..d1a6174 100644
--- a/minifi-commons/minifi-commons-schema/src/test/java/org/apache/nifi/minifi/commons/schema/RemoteProcessGroupSchemaTest.java
+++ b/minifi-commons/minifi-commons-schema/src/test/java/org/apache/nifi/minifi/commons/schema/RemoteProcessGroupSchemaTest.java
@@ -73,6 +73,18 @@ public class RemoteProcessGroupSchemaTest {
     }
 
     @Test
+    public void testLocalNetworkInterface() {
+        Map<String, Object> map = new HashMap<>();
+        map.put(CommonPropertyKeys.INPUT_PORTS_KEY, Arrays.asList(createPortSchema("f94d2469-39f8-4f07-a0d8-acd9396f639e", "testName", ConfigSchema.TOP_LEVEL_NAME).toMap()));
+        map.put(RemoteProcessGroupSchema.URL_KEY, "http://localhost:8080/nifi");
+        map.put(CommonPropertyKeys.ID_KEY, "a58d2fab-7efe-4cb7-8224-12a60bd8003d");
+
+        map.put(RemoteProcessGroupSchema.LOCAL_NETWORK_INTERFACE_KEY, "eth1");
+        RemoteProcessGroupSchema first =  new RemoteProcessGroupSchema(map);
+        validateIssuesNumMatches(0,first);
+        assertEquals(first.getLocalNetworkInterface(), "eth1");
+    }
+    @Test
     public void testProxySettings() {
         Map<String, Object> map = new HashMap<>();
         map.put(RemoteProcessGroupSchema.PROXY_PORT_KEY, 1234);

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/b5561877/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-nar-utils/src/main/java/org/apache/nifi/init/ConfigurableComponentInitializer.java
----------------------------------------------------------------------
diff --git a/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-nar-utils/src/main/java/org/apache/nifi/init/ConfigurableComponentInitializer.java b/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-nar-utils/src/main/java/org/apache/nifi/init/ConfigurableComponentInitializer.java
new file mode 100644
index 0000000..9a1149c
--- /dev/null
+++ b/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-nar-utils/src/main/java/org/apache/nifi/init/ConfigurableComponentInitializer.java
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.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-minifi/blob/b5561877/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-nar-utils/src/main/java/org/apache/nifi/init/ConfigurableComponentInitializerFactory.java
----------------------------------------------------------------------
diff --git a/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-nar-utils/src/main/java/org/apache/nifi/init/ConfigurableComponentInitializerFactory.java b/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-nar-utils/src/main/java/org/apache/nifi/init/ConfigurableComponentInitializerFactory.java
new file mode 100644
index 0000000..f6ab922
--- /dev/null
+++ b/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-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-minifi/blob/b5561877/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-nar-utils/src/main/java/org/apache/nifi/init/ControllerServiceInitializer.java
----------------------------------------------------------------------
diff --git a/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-nar-utils/src/main/java/org/apache/nifi/init/ControllerServiceInitializer.java b/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-nar-utils/src/main/java/org/apache/nifi/init/ControllerServiceInitializer.java
new file mode 100644
index 0000000..21b107f
--- /dev/null
+++ b/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-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-minifi/blob/b5561877/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-nar-utils/src/main/java/org/apache/nifi/init/ProcessorInitializer.java
----------------------------------------------------------------------
diff --git a/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-nar-utils/src/main/java/org/apache/nifi/init/ProcessorInitializer.java b/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-nar-utils/src/main/java/org/apache/nifi/init/ProcessorInitializer.java
new file mode 100644
index 0000000..06fdead
--- /dev/null
+++ b/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-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-minifi/blob/b5561877/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-nar-utils/src/main/java/org/apache/nifi/init/ReflectionUtils.java
----------------------------------------------------------------------
diff --git a/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-nar-utils/src/main/java/org/apache/nifi/init/ReflectionUtils.java b/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-nar-utils/src/main/java/org/apache/nifi/init/ReflectionUtils.java
new file mode 100644
index 0000000..22420bd
--- /dev/null
+++ b/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-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-minifi/blob/b5561877/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-nar-utils/src/main/java/org/apache/nifi/init/ReportingTaskingInitializer.java
----------------------------------------------------------------------
diff --git a/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-nar-utils/src/main/java/org/apache/nifi/init/ReportingTaskingInitializer.java b/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-nar-utils/src/main/java/org/apache/nifi/init/ReportingTaskingInitializer.java
new file mode 100644
index 0000000..f0f495d
--- /dev/null
+++ b/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-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-minifi/blob/b5561877/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-nar-utils/src/main/java/org/apache/nifi/mock/MockComponentLogger.java
----------------------------------------------------------------------
diff --git a/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-nar-utils/src/main/java/org/apache/nifi/mock/MockComponentLogger.java b/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-nar-utils/src/main/java/org/apache/nifi/mock/MockComponentLogger.java
new file mode 100644
index 0000000..920d7eb
--- /dev/null
+++ b/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-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-minifi/blob/b5561877/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-nar-utils/src/main/java/org/apache/nifi/mock/MockConfigurationContext.java
----------------------------------------------------------------------
diff --git a/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-nar-utils/src/main/java/org/apache/nifi/mock/MockConfigurationContext.java b/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-nar-utils/src/main/java/org/apache/nifi/mock/MockConfigurationContext.java
new file mode 100644
index 0000000..d1e73fb
--- /dev/null
+++ b/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-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-minifi/blob/b5561877/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-nar-utils/src/main/java/org/apache/nifi/mock/MockControllerServiceInitializationContext.java
----------------------------------------------------------------------
diff --git a/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-nar-utils/src/main/java/org/apache/nifi/mock/MockControllerServiceInitializationContext.java b/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-nar-utils/src/main/java/org/apache/nifi/mock/MockControllerServiceInitializationContext.java
new file mode 100644
index 0000000..b111ad2
--- /dev/null
+++ b/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-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-minifi/blob/b5561877/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-nar-utils/src/main/java/org/apache/nifi/mock/MockControllerServiceLookup.java
----------------------------------------------------------------------
diff --git a/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-nar-utils/src/main/java/org/apache/nifi/mock/MockControllerServiceLookup.java b/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-nar-utils/src/main/java/org/apache/nifi/mock/MockControllerServiceLookup.java
new file mode 100644
index 0000000..5307ac4
--- /dev/null
+++ b/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-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-minifi/blob/b5561877/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-nar-utils/src/main/java/org/apache/nifi/mock/MockNodeTypeProvider.java
----------------------------------------------------------------------
diff --git a/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-nar-utils/src/main/java/org/apache/nifi/mock/MockNodeTypeProvider.java b/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-nar-utils/src/main/java/org/apache/nifi/mock/MockNodeTypeProvider.java
new file mode 100644
index 0000000..61390e1
--- /dev/null
+++ b/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-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-minifi/blob/b5561877/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-nar-utils/src/main/java/org/apache/nifi/mock/MockProcessContext.java
----------------------------------------------------------------------
diff --git a/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-nar-utils/src/main/java/org/apache/nifi/mock/MockProcessContext.java b/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-nar-utils/src/main/java/org/apache/nifi/mock/MockProcessContext.java
new file mode 100644
index 0000000..cf2e2cf
--- /dev/null
+++ b/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-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-minifi/blob/b5561877/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-nar-utils/src/main/java/org/apache/nifi/mock/MockProcessorInitializationContext.java
----------------------------------------------------------------------
diff --git a/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-nar-utils/src/main/java/org/apache/nifi/mock/MockProcessorInitializationContext.java b/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-nar-utils/src/main/java/org/apache/nifi/mock/MockProcessorInitializationContext.java
new file mode 100644
index 0000000..d9320b2
--- /dev/null
+++ b/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-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-minifi/blob/b5561877/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-nar-utils/src/main/java/org/apache/nifi/mock/MockReportingInitializationContext.java
----------------------------------------------------------------------
diff --git a/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-nar-utils/src/main/java/org/apache/nifi/mock/MockReportingInitializationContext.java b/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-nar-utils/src/main/java/org/apache/nifi/mock/MockReportingInitializationContext.java
new file mode 100644
index 0000000..630c657
--- /dev/null
+++ b/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-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;
+    }
+}