You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@nifi.apache.org by bb...@apache.org on 2018/12/10 16:16:14 UTC

[2/2] nifi git commit: NIFI-5859: Added XML-based documentation writer that can be used to document a component. - Found several instances of nifi-framework-api's ProviderException being thrown from processors. Changed those to IllegalStateException, as

NIFI-5859: Added XML-based documentation writer that can be used to document a component.
- Found several instances of nifi-framework-api's ProviderException being thrown from processors. Changed those to IllegalStateException, as ProviderException is not an appropriate Exception in those cases, and extensions should not depend on nifi-framework-api.
- Performed some cleanup, moving Property Descriptors from Controller Service API's/specs into the implementations. Adding to the Service API results in bringing in nifi-utils to the nifi-standard-services-api-nar, which is a bad practice. The 'main' service api nar should not have a dependency on a util class.

NIFI-5859: Added javadocs. Fixed pom.xml that was left pointing to snapshot version of nar maven plugin

NIFI-5859: Addressing review feedback: adding component type, multiple additional details into separate file(s)

This closes #3192.

Signed-off-by: Bryan Bende <bb...@apache.org>


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

Branch: refs/heads/master
Commit: 1ac5b93144ebd92c907fb49664a524e196ae20c6
Parents: 4e7a856
Author: Mark Payne <ma...@hotmail.com>
Authored: Thu Nov 15 12:07:08 2018 -0500
Committer: Bryan Bende <bb...@apache.org>
Committed: Mon Dec 10 11:16:01 2018 -0500

----------------------------------------------------------------------
 .../AbstractDocumentationWriter.java            | 285 +++++++++++++
 .../ExtensionDocumentationWriter.java           |  42 ++
 .../nifi/documentation/ExtensionType.java       |  25 ++
 .../nifi/documentation/ProvidedServiceAPI.java  |  51 +++
 .../StandardProvidedServiceAPI.java             |  51 +++
 ...nControllerServiceInitializationContext.java |  66 +++
 ...mentationProcessorInitializationContext.java |  65 +++
 ...mentationReportingInitializationContext.java |  89 ++++
 .../init/EmptyControllerServiceLookup.java      |  54 +++
 .../documentation/init/NopComponentLog.java     | 172 ++++++++
 .../documentation/init/NopStateManager.java     |  43 ++
 .../init/StandaloneNodeTypeProvider.java        |  31 ++
 .../xml/XmlDocumentationWriter.java             | 407 +++++++++++++++++++
 .../amqp/processors/AbstractAMQPProcessor.java  |  25 +-
 .../processors/AbstractAMQPProcessorTest.java   |  12 +-
 .../nifi-druid-controller-service-api/pom.xml   |   1 -
 .../pom.xml                                     |   1 -
 .../nifi-elasticsearch-client-service/pom.xml   |   1 -
 .../apache/nifi/documentation/DocGenerator.java |   2 +-
 .../html/HtmlDocumentationWriterTest.java       |   2 +-
 .../html/ProcessorDocumentationWriterTest.java  |  12 +-
 .../org/apache/nifi/web/server/JettyServer.java |   5 +-
 .../nifi-groovyx-processors/pom.xml             |   1 -
 .../mongodb/AbstractMongoProcessor.java         |   3 +-
 .../mongodb/AbstractMongoProcessorTest.java     |  20 +-
 .../nifi/mongodb/MongoDBControllerService.java  |   3 +-
 .../nifi-hbase-client-service-api/pom.xml       |   5 -
 .../apache/nifi/hbase/HBaseClientService.java   |  50 ---
 .../hbase/validate/ConfigFilesValidator.java    |  38 --
 .../nifi/hbase/HBase_1_1_2_ClientService.java   |  47 +++
 .../hbase/validate/ConfigFilesValidator.java    |  38 ++
 .../nifi-standard-services-api-nar/pom.xml      |   6 +
 pom.xml                                         |  74 ++--
 33 files changed, 1548 insertions(+), 179 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/nifi/blob/1ac5b931/nifi-api/src/main/java/org/apache/nifi/documentation/AbstractDocumentationWriter.java
----------------------------------------------------------------------
diff --git a/nifi-api/src/main/java/org/apache/nifi/documentation/AbstractDocumentationWriter.java b/nifi-api/src/main/java/org/apache/nifi/documentation/AbstractDocumentationWriter.java
new file mode 100644
index 0000000..c6e793b
--- /dev/null
+++ b/nifi-api/src/main/java/org/apache/nifi/documentation/AbstractDocumentationWriter.java
@@ -0,0 +1,285 @@
+/*
+ * 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.documentation;
+
+import org.apache.nifi.annotation.behavior.DynamicProperties;
+import org.apache.nifi.annotation.behavior.DynamicProperty;
+import org.apache.nifi.annotation.behavior.DynamicRelationship;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.ReadsAttribute;
+import org.apache.nifi.annotation.behavior.ReadsAttributes;
+import org.apache.nifi.annotation.behavior.Restricted;
+import org.apache.nifi.annotation.behavior.Stateful;
+import org.apache.nifi.annotation.behavior.SystemResourceConsideration;
+import org.apache.nifi.annotation.behavior.WritesAttribute;
+import org.apache.nifi.annotation.behavior.WritesAttributes;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.DeprecationNotice;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.components.ConfigurableComponent;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.controller.ControllerService;
+import org.apache.nifi.documentation.init.DocumentationControllerServiceInitializationContext;
+import org.apache.nifi.documentation.init.DocumentationProcessorInitializationContext;
+import org.apache.nifi.documentation.init.DocumentationReportingInitializationContext;
+import org.apache.nifi.processor.Processor;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.reporting.InitializationException;
+import org.apache.nifi.reporting.ReportingTask;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+
+/**
+ * Base class for DocumentationWriter that simplifies iterating over all information for a component, creating a separate method
+ * for each, to ensure that implementations properly override all methods and therefore properly account for all information about
+ * a component.
+ *
+ * Please note that while this class lives within the nifi-api, it is provided primarily as a means for documentation components within
+ * the NiFi NAR Maven Plugin. Its home is the nifi-api, however, because the API is needed in order to extract the relevant information and
+ * the NAR Maven Plugin cannot have a direct dependency on nifi-api (doing so would cause a circular dependency). By having this homed within
+ * the nifi-api, the Maven plugin is able to discover the class dynamically and invoke the one or two methods necessary to create the documentation.
+ *
+ * This is a new capability in 1.9.0 in preparation for the Extension Registry and therefore, you should
+ * <b>NOTE WELL:</b> At this time, while this class is part of nifi-api, it is still evolving and may change in a non-backward-compatible manner or even be
+ * removed from one incremental release to the next. Use at your own risk!
+ */
+public abstract class AbstractDocumentationWriter implements ExtensionDocumentationWriter {
+
+    @Override
+    public final void write(final ConfigurableComponent component) throws IOException {
+        write(component, null);
+    }
+
+    @Override
+    public final void write(final ConfigurableComponent component, final Collection<ProvidedServiceAPI> providedServices) throws IOException {
+        initialize(component);
+
+        writeHeader(component);
+        writeBody(component);
+
+        if (providedServices != null && component instanceof ControllerService) {
+            writeProvidedServices(providedServices);
+        }
+
+        writeFooter(component);
+    }
+
+    private void initialize(final ConfigurableComponent component) {
+        try {
+            if (component instanceof Processor) {
+                initialize((Processor) component);
+            } else if (component instanceof ControllerService) {
+                initialize((ControllerService) component);
+            } else if (component instanceof ReportingTask) {
+                initialize((ReportingTask) component);
+            }
+        } catch (final InitializationException ie) {
+            throw new RuntimeException("Failed to initialize " + component, ie);
+        }
+    }
+
+    protected void initialize(final Processor processor) {
+        processor.initialize(new DocumentationProcessorInitializationContext());
+    }
+
+    protected void initialize(final ControllerService service) throws InitializationException {
+        service.initialize(new DocumentationControllerServiceInitializationContext());
+    }
+
+    protected void initialize(final ReportingTask reportingTask) throws InitializationException {
+        reportingTask.initialize(new DocumentationReportingInitializationContext());
+    }
+
+    protected void writeBody(final ConfigurableComponent component) throws IOException {
+        writeExtensionName(component.getClass().getName());
+        writeExtensionType(getExtensionType(component));
+        writeDeprecationNotice(component.getClass().getAnnotation(DeprecationNotice.class));
+        writeDescription(getDescription(component));
+        writeTags(getTags(component));
+        writeProperties(component.getPropertyDescriptors());
+        writeDynamicProperties(getDynamicProperties(component));
+
+        if (component instanceof Processor) {
+            final Processor processor = (Processor) component;
+
+            writeRelationships(processor.getRelationships());
+            writeDynamicRelationship(getDynamicRelationship(processor));
+            writeReadsAttributes(getReadsAttributes(processor));
+            writeWritesAttributes(getWritesAttributes(processor));
+        }
+
+        writeStatefulInfo(component.getClass().getAnnotation(Stateful.class));
+        writeRestrictedInfo(component.getClass().getAnnotation(Restricted.class));
+        writeInputRequirementInfo(getInputRequirement(component));
+        writeSystemResourceConsiderationInfo(getSystemResourceConsiderations(component));
+        writeSeeAlso(component.getClass().getAnnotation(SeeAlso.class));
+    }
+
+
+    protected String getDescription(final ConfigurableComponent component) {
+        final CapabilityDescription capabilityDescription = component.getClass().getAnnotation(CapabilityDescription.class);
+        if (capabilityDescription == null) {
+            return null;
+        }
+
+        return capabilityDescription.value();
+    }
+
+    protected List<String> getTags(final ConfigurableComponent component) {
+        final Tags tags = component.getClass().getAnnotation(Tags.class);
+        if (tags == null) {
+            return Collections.emptyList();
+        }
+
+        final String[] tagValues = tags.value();
+        return tagValues == null ? Collections.emptyList() : Arrays.asList(tagValues);
+    }
+
+    protected List<DynamicProperty> getDynamicProperties(ConfigurableComponent configurableComponent) {
+        final List<DynamicProperty> dynamicProperties = new ArrayList<>();
+        final DynamicProperties dynProps = configurableComponent.getClass().getAnnotation(DynamicProperties.class);
+        if (dynProps != null) {
+            Collections.addAll(dynamicProperties, dynProps.value());
+        }
+
+        final DynamicProperty dynProp = configurableComponent.getClass().getAnnotation(DynamicProperty.class);
+        if (dynProp != null) {
+            dynamicProperties.add(dynProp);
+        }
+
+        return dynamicProperties;
+    }
+
+
+    private DynamicRelationship getDynamicRelationship(Processor processor) {
+        return processor.getClass().getAnnotation(DynamicRelationship.class);
+    }
+
+
+    private List<ReadsAttribute> getReadsAttributes(final Processor processor) {
+        final List<ReadsAttribute> attributes = new ArrayList<>();
+
+        final ReadsAttributes readsAttributes = processor.getClass().getAnnotation(ReadsAttributes.class);
+        if (readsAttributes != null) {
+            Collections.addAll(attributes, readsAttributes.value());
+        }
+
+        final ReadsAttribute readsAttribute = processor.getClass().getAnnotation(ReadsAttribute.class);
+        if (readsAttribute != null) {
+            attributes.add(readsAttribute);
+        }
+
+        return attributes;
+    }
+
+
+    private List<WritesAttribute> getWritesAttributes(Processor processor) {
+        List<WritesAttribute> attributes = new ArrayList<>();
+
+        WritesAttributes writesAttributes = processor.getClass().getAnnotation(WritesAttributes.class);
+        if (writesAttributes != null) {
+            Collections.addAll(attributes, writesAttributes.value());
+        }
+
+        WritesAttribute writeAttribute = processor.getClass().getAnnotation(WritesAttribute.class);
+        if (writeAttribute != null) {
+            attributes.add(writeAttribute);
+        }
+
+        return attributes;
+    }
+
+    private InputRequirement.Requirement getInputRequirement(final ConfigurableComponent component) {
+        final InputRequirement annotation = component.getClass().getAnnotation(InputRequirement.class);
+        return annotation == null ? null : annotation.value();
+    }
+
+    private List<SystemResourceConsideration> getSystemResourceConsiderations(final ConfigurableComponent component) {
+        SystemResourceConsideration[] systemResourceConsiderations = component.getClass().getAnnotationsByType(SystemResourceConsideration.class);
+        if (systemResourceConsiderations == null) {
+            return Collections.emptyList();
+        }
+
+        return Arrays.asList(systemResourceConsiderations);
+    }
+
+    protected ExtensionType getExtensionType(final ConfigurableComponent component) {
+        if (component instanceof Processor) {
+            return ExtensionType.PROCESSOR;
+        }
+        if (component instanceof ControllerService) {
+            return ExtensionType.CONTROLLER_SERVICE;
+        }
+        if (component instanceof ReportingTask) {
+            return ExtensionType.REPORTING_TASK;
+        }
+        throw new AssertionError("Encountered unknown Configurable Component Type for " + component);
+    }
+
+
+    protected abstract void writeHeader(ConfigurableComponent component) throws IOException;
+
+    protected abstract void writeExtensionName(String extensionName) throws IOException;
+
+    protected abstract void writeExtensionType(ExtensionType extensionType) throws IOException;
+
+    protected abstract void writeDeprecationNotice(final DeprecationNotice deprecationNotice) throws IOException;
+
+
+    protected abstract void writeDescription(String description) throws IOException;
+
+    protected abstract void writeTags(List<String> tags) throws IOException;
+
+    protected abstract void writeProperties(List<PropertyDescriptor> properties) throws IOException;
+
+    protected abstract void writeDynamicProperties(List<DynamicProperty> dynamicProperties) throws IOException;
+
+    protected abstract void writeStatefulInfo(Stateful stateful) throws IOException;
+
+    protected abstract void writeRestrictedInfo(Restricted restricted) throws IOException;
+
+    protected abstract void writeInputRequirementInfo(InputRequirement.Requirement requirement) throws IOException;
+
+    protected abstract void writeSystemResourceConsiderationInfo(List<SystemResourceConsideration> considerations) throws IOException;
+
+    protected abstract void writeSeeAlso(SeeAlso seeAlso) throws IOException;
+
+
+
+    // Processor-specific methods
+    protected abstract void writeRelationships(Set<Relationship> relationships) throws IOException;
+
+    protected abstract void writeDynamicRelationship(DynamicRelationship dynamicRelationship) throws IOException;
+
+    protected abstract void writeReadsAttributes(List<ReadsAttribute> attributes) throws IOException;
+
+    protected abstract void writeWritesAttributes(List<WritesAttribute> attributes) throws IOException;
+
+
+    // ControllerService-specific methods
+    protected abstract void writeProvidedServices(Collection<ProvidedServiceAPI> providedServices) throws IOException;
+
+
+    protected abstract void writeFooter(ConfigurableComponent component) throws IOException;
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/1ac5b931/nifi-api/src/main/java/org/apache/nifi/documentation/ExtensionDocumentationWriter.java
----------------------------------------------------------------------
diff --git a/nifi-api/src/main/java/org/apache/nifi/documentation/ExtensionDocumentationWriter.java b/nifi-api/src/main/java/org/apache/nifi/documentation/ExtensionDocumentationWriter.java
new file mode 100644
index 0000000..c533e4e
--- /dev/null
+++ b/nifi-api/src/main/java/org/apache/nifi/documentation/ExtensionDocumentationWriter.java
@@ -0,0 +1,42 @@
+/*
+ * 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.documentation;
+
+import org.apache.nifi.components.ConfigurableComponent;
+
+import java.io.IOException;
+import java.util.Collection;
+
+/**
+ * Generates documentation for an instance of a ConfigurableComponent.
+ *
+ * Please note that while this class lives within the nifi-api, it is provided primarily as a means for documentation components within
+ * the NiFi NAR Maven Plugin. Its home is the nifi-api, however, because the API is needed in order to extract the relevant information and
+ * the NAR Maven Plugin cannot have a direct dependency on nifi-api (doing so would cause a circular dependency). By having this homed within
+ * the nifi-api, the Maven plugin is able to discover the class dynamically and invoke the one or two methods necessary to create the documentation.
+ *
+ * This is a new capability in 1.9.0 in preparation for the Extension Registry and therefore, you should
+ * <b>NOTE WELL:</b> At this time, while this class is part of nifi-api, it is still evolving and may change in a non-backward-compatible manner or even be
+ * removed from one incremental release to the next. Use at your own risk!
+ */
+public interface ExtensionDocumentationWriter {
+
+    void write(ConfigurableComponent component) throws IOException;
+
+    void write(ConfigurableComponent component, Collection<ProvidedServiceAPI> provideServices) throws IOException;
+
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/1ac5b931/nifi-api/src/main/java/org/apache/nifi/documentation/ExtensionType.java
----------------------------------------------------------------------
diff --git a/nifi-api/src/main/java/org/apache/nifi/documentation/ExtensionType.java b/nifi-api/src/main/java/org/apache/nifi/documentation/ExtensionType.java
new file mode 100644
index 0000000..f986939
--- /dev/null
+++ b/nifi-api/src/main/java/org/apache/nifi/documentation/ExtensionType.java
@@ -0,0 +1,25 @@
+/*
+ * 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.documentation;
+
+public enum ExtensionType {
+    PROCESSOR,
+
+    CONTROLLER_SERVICE,
+
+    REPORTING_TASK;
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/1ac5b931/nifi-api/src/main/java/org/apache/nifi/documentation/ProvidedServiceAPI.java
----------------------------------------------------------------------
diff --git a/nifi-api/src/main/java/org/apache/nifi/documentation/ProvidedServiceAPI.java b/nifi-api/src/main/java/org/apache/nifi/documentation/ProvidedServiceAPI.java
new file mode 100644
index 0000000..04cd425
--- /dev/null
+++ b/nifi-api/src/main/java/org/apache/nifi/documentation/ProvidedServiceAPI.java
@@ -0,0 +1,51 @@
+/*
+ * 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.documentation;
+
+/**
+ * Describes a Controller Service API that is provided by some implementation.
+ *
+ * Please note that while this class lives within the nifi-api, it is provided primarily as a means for documentation components within
+ * the NiFi NAR Maven Plugin. Its home is the nifi-api, however, because the API is needed in order to extract the relevant information and
+ * the NAR Maven Plugin cannot have a direct dependency on nifi-api (doing so would cause a circular dependency). By having this homed within
+ * the nifi-api, the Maven plugin is able to discover the class dynamically and invoke the one or two methods necessary to create the documentation.
+ *
+ * This is a new capability in 1.9.0 in preparation for the Extension Registry and therefore, you should
+ * <b>NOTE WELL:</b> At this time, while this class is part of nifi-api, it is still evolving and may change in a non-backward-compatible manner or even be
+ * removed from one incremental release to the next. Use at your own risk!
+ */
+public interface ProvidedServiceAPI {
+    /**
+     * @return the fully qualified class name of the interface implemented by the Controller Service
+     */
+    String getClassName();
+
+    /**
+     * @return the Group ID of the bundle that provides the interface
+     */
+    String getGroupId();
+
+    /**
+     * @return the Artifact ID of the bundle that provides the interface
+     */
+    String getArtifactId();
+
+    /**
+     * @return the Version of the bundle that provides the interface
+     */
+    String getVersion();
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/1ac5b931/nifi-api/src/main/java/org/apache/nifi/documentation/StandardProvidedServiceAPI.java
----------------------------------------------------------------------
diff --git a/nifi-api/src/main/java/org/apache/nifi/documentation/StandardProvidedServiceAPI.java b/nifi-api/src/main/java/org/apache/nifi/documentation/StandardProvidedServiceAPI.java
new file mode 100644
index 0000000..b86f4ca
--- /dev/null
+++ b/nifi-api/src/main/java/org/apache/nifi/documentation/StandardProvidedServiceAPI.java
@@ -0,0 +1,51 @@
+/*
+ * 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.documentation;
+
+public class StandardProvidedServiceAPI implements ProvidedServiceAPI {
+    private final String className;
+    private final String groupId;
+    private final String artifactId;
+    private final String version;
+
+    public StandardProvidedServiceAPI(final String className, final String groupId, final String artifactId, final String version) {
+        this.className = className;
+        this.groupId = groupId;
+        this.artifactId = artifactId;
+        this.version = version;
+    }
+
+    @Override
+    public String getClassName() {
+        return className;
+    }
+
+    @Override
+    public String getGroupId() {
+        return groupId;
+    }
+
+    @Override
+    public String getArtifactId() {
+        return artifactId;
+    }
+
+    @Override
+    public String getVersion() {
+        return version;
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/1ac5b931/nifi-api/src/main/java/org/apache/nifi/documentation/init/DocumentationControllerServiceInitializationContext.java
----------------------------------------------------------------------
diff --git a/nifi-api/src/main/java/org/apache/nifi/documentation/init/DocumentationControllerServiceInitializationContext.java b/nifi-api/src/main/java/org/apache/nifi/documentation/init/DocumentationControllerServiceInitializationContext.java
new file mode 100644
index 0000000..cdf1e8d
--- /dev/null
+++ b/nifi-api/src/main/java/org/apache/nifi/documentation/init/DocumentationControllerServiceInitializationContext.java
@@ -0,0 +1,66 @@
+/*
+ * 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.documentation.init;
+
+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;
+import java.util.UUID;
+
+public class DocumentationControllerServiceInitializationContext implements ControllerServiceInitializationContext {
+    private final String id = UUID.randomUUID().toString();
+    private final ControllerServiceLookup serviceLookup = new EmptyControllerServiceLookup();
+    private final ComponentLog componentLog = new NopComponentLog();
+
+    @Override
+    public String getIdentifier() {
+        return id;
+    }
+
+    @Override
+    public ControllerServiceLookup getControllerServiceLookup() {
+        return serviceLookup;
+    }
+
+    @Override
+    public ComponentLog getLogger() {
+        return componentLog;
+    }
+
+    @Override
+    public StateManager getStateManager() {
+        return new NopStateManager();
+    }
+
+    @Override
+    public String getKerberosServicePrincipal() {
+        return null;
+    }
+
+    @Override
+    public File getKerberosServiceKeytab() {
+        return null;
+    }
+
+    @Override
+    public File getKerberosConfigurationFile() {
+        return null;
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/1ac5b931/nifi-api/src/main/java/org/apache/nifi/documentation/init/DocumentationProcessorInitializationContext.java
----------------------------------------------------------------------
diff --git a/nifi-api/src/main/java/org/apache/nifi/documentation/init/DocumentationProcessorInitializationContext.java b/nifi-api/src/main/java/org/apache/nifi/documentation/init/DocumentationProcessorInitializationContext.java
new file mode 100644
index 0000000..c7a5e40
--- /dev/null
+++ b/nifi-api/src/main/java/org/apache/nifi/documentation/init/DocumentationProcessorInitializationContext.java
@@ -0,0 +1,65 @@
+/*
+ * 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.documentation.init;
+
+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;
+import java.util.UUID;
+
+public class DocumentationProcessorInitializationContext implements ProcessorInitializationContext {
+    private final String uuid = UUID.randomUUID().toString();
+    private final NodeTypeProvider nodeTypeProvider = new StandaloneNodeTypeProvider();
+
+    @Override
+    public String getIdentifier() {
+        return uuid;
+    }
+
+    @Override
+    public ComponentLog getLogger() {
+        return new NopComponentLog();
+    }
+
+    @Override
+    public ControllerServiceLookup getControllerServiceLookup() {
+        return new EmptyControllerServiceLookup();
+    }
+
+    @Override
+    public NodeTypeProvider getNodeTypeProvider() {
+        return nodeTypeProvider;
+    }
+
+    @Override
+    public String getKerberosServicePrincipal() {
+        return null;
+    }
+
+    @Override
+    public File getKerberosServiceKeytab() {
+        return null;
+    }
+
+    @Override
+    public File getKerberosConfigurationFile() {
+        return null;
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/1ac5b931/nifi-api/src/main/java/org/apache/nifi/documentation/init/DocumentationReportingInitializationContext.java
----------------------------------------------------------------------
diff --git a/nifi-api/src/main/java/org/apache/nifi/documentation/init/DocumentationReportingInitializationContext.java b/nifi-api/src/main/java/org/apache/nifi/documentation/init/DocumentationReportingInitializationContext.java
new file mode 100644
index 0000000..4697ee8
--- /dev/null
+++ b/nifi-api/src/main/java/org/apache/nifi/documentation/init/DocumentationReportingInitializationContext.java
@@ -0,0 +1,89 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.documentation.init;
+
+import org.apache.nifi.controller.ControllerServiceLookup;
+import org.apache.nifi.controller.NodeTypeProvider;
+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.UUID;
+import java.util.concurrent.TimeUnit;
+
+public class DocumentationReportingInitializationContext implements ReportingInitializationContext {
+    private final String id = UUID.randomUUID().toString();
+    private final ComponentLog componentLog = new NopComponentLog();
+    private final NodeTypeProvider nodeTypeProvider = new StandaloneNodeTypeProvider();
+    private final String name = "name";
+
+    @Override
+    public String getIdentifier() {
+        return id;
+    }
+
+    @Override
+    public String getName() {
+        return name;
+    }
+
+    @Override
+    public long getSchedulingPeriod(final TimeUnit timeUnit) {
+        return 0;
+    }
+
+    @Override
+    public ControllerServiceLookup getControllerServiceLookup() {
+        return new EmptyControllerServiceLookup();
+    }
+
+    @Override
+    public String getSchedulingPeriod() {
+        return "0 sec";
+    }
+
+    @Override
+    public SchedulingStrategy getSchedulingStrategy() {
+        return SchedulingStrategy.TIMER_DRIVEN;
+    }
+
+    @Override
+    public ComponentLog getLogger() {
+        return componentLog;
+    }
+
+    @Override
+    public NodeTypeProvider getNodeTypeProvider() {
+        return nodeTypeProvider;
+    }
+
+    @Override
+    public String getKerberosServicePrincipal() {
+        return null;
+    }
+
+    @Override
+    public File getKerberosServiceKeytab() {
+        return null;
+    }
+
+    @Override
+    public File getKerberosConfigurationFile() {
+        return null;
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/1ac5b931/nifi-api/src/main/java/org/apache/nifi/documentation/init/EmptyControllerServiceLookup.java
----------------------------------------------------------------------
diff --git a/nifi-api/src/main/java/org/apache/nifi/documentation/init/EmptyControllerServiceLookup.java b/nifi-api/src/main/java/org/apache/nifi/documentation/init/EmptyControllerServiceLookup.java
new file mode 100644
index 0000000..4831198
--- /dev/null
+++ b/nifi-api/src/main/java/org/apache/nifi/documentation/init/EmptyControllerServiceLookup.java
@@ -0,0 +1,54 @@
+/*
+ * 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.documentation.init;
+
+import org.apache.nifi.controller.ControllerService;
+import org.apache.nifi.controller.ControllerServiceLookup;
+
+import java.util.Set;
+
+public class EmptyControllerServiceLookup implements ControllerServiceLookup {
+    @Override
+    public ControllerService getControllerService(final String serviceIdentifier) {
+        return null;
+    }
+
+    @Override
+    public boolean isControllerServiceEnabled(final String serviceIdentifier) {
+        return false;
+    }
+
+    @Override
+    public boolean isControllerServiceEnabling(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 null;
+    }
+
+    @Override
+    public String getControllerServiceName(final String serviceIdentifier) {
+        return null;
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/1ac5b931/nifi-api/src/main/java/org/apache/nifi/documentation/init/NopComponentLog.java
----------------------------------------------------------------------
diff --git a/nifi-api/src/main/java/org/apache/nifi/documentation/init/NopComponentLog.java b/nifi-api/src/main/java/org/apache/nifi/documentation/init/NopComponentLog.java
new file mode 100644
index 0000000..50ef5ad
--- /dev/null
+++ b/nifi-api/src/main/java/org/apache/nifi/documentation/init/NopComponentLog.java
@@ -0,0 +1,172 @@
+/*
+ * 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.documentation.init;
+
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.logging.LogLevel;
+
+public class NopComponentLog implements ComponentLog {
+    @Override
+    public void warn(final String msg, final Throwable t) {
+
+    }
+
+    @Override
+    public void warn(final String msg, final Object[] os) {
+
+    }
+
+    @Override
+    public void warn(final String msg, final Object[] os, final Throwable t) {
+
+    }
+
+    @Override
+    public void warn(final String msg) {
+
+    }
+
+    @Override
+    public void trace(final String msg, final Throwable t) {
+
+    }
+
+    @Override
+    public void trace(final String msg, final Object[] os) {
+
+    }
+
+    @Override
+    public void trace(final String msg) {
+
+    }
+
+    @Override
+    public void trace(final String msg, final Object[] os, final Throwable t) {
+
+    }
+
+    @Override
+    public boolean isWarnEnabled() {
+        return false;
+    }
+
+    @Override
+    public boolean isTraceEnabled() {
+        return false;
+    }
+
+    @Override
+    public boolean isInfoEnabled() {
+        return false;
+    }
+
+    @Override
+    public boolean isErrorEnabled() {
+        return false;
+    }
+
+    @Override
+    public boolean isDebugEnabled() {
+        return false;
+    }
+
+    @Override
+    public void info(final String msg, final Throwable t) {
+
+    }
+
+    @Override
+    public void info(final String msg, final Object[] os) {
+
+    }
+
+    @Override
+    public void info(final String msg) {
+
+    }
+
+    @Override
+    public void info(final String msg, final Object[] os, final Throwable t) {
+
+    }
+
+    @Override
+    public String getName() {
+        return null;
+    }
+
+    @Override
+    public void error(final String msg, final Throwable t) {
+
+    }
+
+    @Override
+    public void error(final String msg, final Object[] os) {
+
+    }
+
+    @Override
+    public void error(final String msg) {
+
+    }
+
+    @Override
+    public void error(final String msg, final Object[] os, final Throwable t) {
+
+    }
+
+    @Override
+    public void debug(final String msg, final Throwable t) {
+
+    }
+
+    @Override
+    public void debug(final String msg, final Object[] os) {
+
+    }
+
+    @Override
+    public void debug(final String msg, final Object[] os, final Throwable t) {
+
+    }
+
+    @Override
+    public void debug(final String msg) {
+
+    }
+
+    @Override
+    public void log(final LogLevel level, final String msg, final Throwable t) {
+
+    }
+
+    @Override
+    public void log(final LogLevel level, final String msg, final Object[] os) {
+
+    }
+
+    @Override
+    public void log(final LogLevel level, final String msg) {
+
+    }
+
+    @Override
+    public void log(final LogLevel level, final String msg, final Object[] os, final Throwable t) {
+
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/1ac5b931/nifi-api/src/main/java/org/apache/nifi/documentation/init/NopStateManager.java
----------------------------------------------------------------------
diff --git a/nifi-api/src/main/java/org/apache/nifi/documentation/init/NopStateManager.java b/nifi-api/src/main/java/org/apache/nifi/documentation/init/NopStateManager.java
new file mode 100644
index 0000000..5e2c955
--- /dev/null
+++ b/nifi-api/src/main/java/org/apache/nifi/documentation/init/NopStateManager.java
@@ -0,0 +1,43 @@
+/*
+ * 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.documentation.init;
+
+import org.apache.nifi.components.state.Scope;
+import org.apache.nifi.components.state.StateManager;
+import org.apache.nifi.components.state.StateMap;
+
+import java.util.Map;
+
+public class NopStateManager implements StateManager {
+    @Override
+    public void setState(final Map<String, String> state, final Scope scope) {
+    }
+
+    @Override
+    public StateMap getState(final Scope scope) {
+        return null;
+    }
+
+    @Override
+    public boolean replace(final StateMap oldValue, final Map<String, String> newValue, final Scope scope) {
+        return false;
+    }
+
+    @Override
+    public void clear(final Scope scope) {
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/1ac5b931/nifi-api/src/main/java/org/apache/nifi/documentation/init/StandaloneNodeTypeProvider.java
----------------------------------------------------------------------
diff --git a/nifi-api/src/main/java/org/apache/nifi/documentation/init/StandaloneNodeTypeProvider.java b/nifi-api/src/main/java/org/apache/nifi/documentation/init/StandaloneNodeTypeProvider.java
new file mode 100644
index 0000000..a0c5be6
--- /dev/null
+++ b/nifi-api/src/main/java/org/apache/nifi/documentation/init/StandaloneNodeTypeProvider.java
@@ -0,0 +1,31 @@
+/*
+ * 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.documentation.init;
+
+import org.apache.nifi.controller.NodeTypeProvider;
+
+public class StandaloneNodeTypeProvider implements NodeTypeProvider {
+    @Override
+    public boolean isClustered() {
+        return false;
+    }
+
+    @Override
+    public boolean isPrimary() {
+        return false;
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/1ac5b931/nifi-api/src/main/java/org/apache/nifi/documentation/xml/XmlDocumentationWriter.java
----------------------------------------------------------------------
diff --git a/nifi-api/src/main/java/org/apache/nifi/documentation/xml/XmlDocumentationWriter.java b/nifi-api/src/main/java/org/apache/nifi/documentation/xml/XmlDocumentationWriter.java
new file mode 100644
index 0000000..93dee90
--- /dev/null
+++ b/nifi-api/src/main/java/org/apache/nifi/documentation/xml/XmlDocumentationWriter.java
@@ -0,0 +1,407 @@
+/*
+ * 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.documentation.xml;
+
+import org.apache.nifi.annotation.behavior.DynamicProperty;
+import org.apache.nifi.annotation.behavior.DynamicRelationship;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.ReadsAttribute;
+import org.apache.nifi.annotation.behavior.Restricted;
+import org.apache.nifi.annotation.behavior.Restriction;
+import org.apache.nifi.annotation.behavior.Stateful;
+import org.apache.nifi.annotation.behavior.SystemResourceConsideration;
+import org.apache.nifi.annotation.behavior.WritesAttribute;
+import org.apache.nifi.annotation.documentation.DeprecationNotice;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+import org.apache.nifi.components.AllowableValue;
+import org.apache.nifi.components.ConfigurableComponent;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.RequiredPermission;
+import org.apache.nifi.documentation.AbstractDocumentationWriter;
+import org.apache.nifi.documentation.ExtensionType;
+import org.apache.nifi.documentation.ProvidedServiceAPI;
+import org.apache.nifi.processor.Relationship;
+
+import javax.xml.stream.XMLOutputFactory;
+import javax.xml.stream.XMLStreamException;
+import javax.xml.stream.XMLStreamWriter;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.LinkedHashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.function.Function;
+
+/**
+ * XML-based implementation of DocumentationWriter
+ *
+ * Please note that while this class lives within the nifi-api, it is provided primarily as a means for documentation components within
+ * the NiFi NAR Maven Plugin. Its home is the nifi-api, however, because the API is needed in order to extract the relevant information and
+ * the NAR Maven Plugin cannot have a direct dependency on nifi-api (doing so would cause a circular dependency). By having this homed within
+ * the nifi-api, the Maven plugin is able to discover the class dynamically and invoke the one or two methods necessary to create the documentation.
+ *
+ * This is a new capability in 1.9.0 in preparation for the Extension Registry and therefore, you should
+ * <b>NOTE WELL:</b> At this time, while this class is part of nifi-api, it is still evolving and may change in a non-backward-compatible manner or even be
+ * removed from one incremental release to the next. Use at your own risk!
+ */
+public class XmlDocumentationWriter extends AbstractDocumentationWriter {
+    private final XMLStreamWriter writer;
+
+    public XmlDocumentationWriter(final OutputStream out) throws XMLStreamException {
+        this.writer = XMLOutputFactory.newInstance().createXMLStreamWriter(out, "UTF-8");
+    }
+
+    public XmlDocumentationWriter(final XMLStreamWriter writer) {
+        this.writer = writer;
+    }
+
+    @Override
+    protected void writeHeader(final ConfigurableComponent component) throws IOException {
+        writeStartElement("extension");
+    }
+
+    @Override
+    protected void writeExtensionName(final String extensionName) throws IOException {
+        writeTextElement("name", extensionName);
+    }
+
+    @Override
+    protected void writeExtensionType(final ExtensionType extensionType) throws IOException {
+        writeTextElement("type", extensionType.name());
+    }
+
+    @Override
+    protected void writeDeprecationNotice(final DeprecationNotice deprecationNotice) throws IOException {
+        if (deprecationNotice == null) {
+            writeEmptyElement("deprecationNotice");
+            return;
+        }
+
+        final Class[] classes = deprecationNotice.alternatives();
+        final String[] classNames = deprecationNotice.classNames();
+
+        final Set<String> alternatives = new LinkedHashSet<>();
+        if (classes != null) {
+            for (final Class alternativeClass : classes) {
+                alternatives.add(alternativeClass.getName());
+            }
+        }
+
+        if (classNames != null) {
+            Collections.addAll(alternatives, classNames);
+        }
+
+        writeDeprecationNotice(deprecationNotice.reason(), alternatives);
+    }
+
+    private void writeDeprecationNotice(final String reason, final Set<String> alternatives) throws IOException {
+        writeStartElement("deprecationNotice");
+
+        writeTextElement("reason", reason);
+        writeTextArray("alternatives", "alternative", alternatives);
+
+        writeEndElement();
+    }
+
+
+    @Override
+    protected void writeDescription(final String description) throws IOException {
+        writeTextElement("description", description);
+    }
+
+    @Override
+    protected void writeTags(final List<String> tags) throws IOException {
+        writeTextArray("tags", "tag", tags);
+    }
+
+    @Override
+    protected void writeProperties(final List<PropertyDescriptor> properties) throws IOException {
+        writeArray("properties", properties, this::writeProperty);
+    }
+
+    private void writeProperty(final PropertyDescriptor property) throws IOException {
+        writeStartElement("property");
+
+        writeTextElement("name", property.getName());
+        writeTextElement("displayName", property.getDisplayName());
+        writeTextElement("description", property.getDescription());
+        writeTextElement("defaultValue", property.getDefaultValue());
+        writeTextElement("controllerServiceDefinition", property.getControllerServiceDefinition() == null ? null : property.getControllerServiceDefinition().getName());
+        writeTextArray("allowableValues", "allowableValue", property.getAllowableValues(), AllowableValue::getDisplayName);
+        writeBooleanElement("required", property.isRequired());
+        writeBooleanElement("sensitive", property.isSensitive());
+        writeBooleanElement("expressionLanguageSupported", property.isExpressionLanguageSupported());
+        writeTextElement("expressionLanguageScope", property.getExpressionLanguageScope() == null ? null : property.getExpressionLanguageScope().name());
+        writeBooleanElement("dynamicallyModifiesClasspath", property.isDynamicClasspathModifier());
+        writeBooleanElement("dynamic", property.isDynamic());
+
+        writeEndElement();
+    }
+
+    @Override
+    protected void writeDynamicProperties(final List<DynamicProperty> dynamicProperties) throws IOException {
+        writeArray("dynamicProperty", dynamicProperties, this::writeDynamicProperty);
+    }
+
+    private void writeDynamicProperty(final DynamicProperty property) throws IOException {
+        writeStartElement("dynamicProperty");
+
+        writeTextElement("name", property.name());
+        writeTextElement("value", property.value());
+        writeTextElement("description", property.description());
+        writeBooleanElement("expressionLanguageSupported", property.supportsExpressionLanguage());
+        writeTextElement("expressionLanguageScope", property.expressionLanguageScope() == null ? null : property.expressionLanguageScope().name());
+
+        writeEndElement();
+    }
+
+    @Override
+    protected void writeStatefulInfo(final Stateful stateful) throws IOException {
+        writeStartElement("stateful");
+
+        if (stateful != null) {
+            writeTextElement("description", stateful.description());
+            writeArray("scopes", Arrays.asList(stateful.scopes()), scope -> writeTextElement("scope", scope.name()));
+        }
+
+        writeEndElement();
+    }
+
+    @Override
+    protected void writeRestrictedInfo(final Restricted restricted) throws IOException {
+        writeStartElement("restricted");
+
+        if (restricted != null) {
+            writeTextElement("generalRestrictionExplanation", restricted.value());
+
+            final Restriction[] restrictions = restricted.restrictions();
+            if (restrictions != null) {
+                writeArray("restrictions", Arrays.asList(restrictions), this::writeRestriction);
+            }
+        }
+
+        writeEndElement();
+    }
+
+    private void writeRestriction(final Restriction restriction) throws IOException {
+        writeStartElement("restriction");
+
+        final RequiredPermission permission = restriction.requiredPermission();
+        final String label = permission == null ? null : permission.getPermissionLabel();
+        writeTextElement("requiredPermission", label);
+        writeTextElement("explanation", restriction.explanation());
+
+        writeEndElement();
+    }
+
+    @Override
+    protected void writeInputRequirementInfo(final InputRequirement.Requirement requirement) throws IOException {
+        writeTextElement("inputRequirement", requirement == null ? null : requirement.name());
+    }
+
+    @Override
+    protected void writeSystemResourceConsiderationInfo(final List<SystemResourceConsideration> considerations) throws IOException {
+        writeArray("systemResourceConsiderations", considerations, this::writeSystemResourceConsideration);
+    }
+
+    private void writeSystemResourceConsideration(final SystemResourceConsideration consideration) throws IOException {
+        writeStartElement("consideration");
+
+        writeTextElement("resource", consideration.resource() == null ? null : consideration.resource().name());
+        writeTextElement("description", consideration.description());
+
+        writeEndElement();
+    }
+
+    @Override
+    protected void writeSeeAlso(final SeeAlso seeAlso) throws IOException {
+        if (seeAlso == null) {
+            writeEmptyElement("seeAlso");
+            return;
+        }
+
+        final Class[] classes = seeAlso.value();
+        final String[] classNames = seeAlso.classNames();
+
+        final Set<String> toSee = new LinkedHashSet<>();
+        if (classes != null) {
+            for (final Class classToSee : classes) {
+                toSee.add(classToSee.getName());
+            }
+        }
+
+        if (classNames != null) {
+            Collections.addAll(toSee, classNames);
+        }
+
+        writeTextArray("seeAlso", "see", toSee);
+    }
+
+    @Override
+    protected void writeRelationships(final Set<Relationship> relationships) throws IOException {
+        writeArray("relationships", relationships,rel -> {
+            writeStartElement("relationship");
+
+            writeTextElement("name", rel.getName());
+            writeTextElement("description", rel.getDescription());
+            writeBooleanElement("autoTerminated", rel.isAutoTerminated());
+
+            writeEndElement();
+        } );
+    }
+
+    @Override
+    protected void writeDynamicRelationship(final DynamicRelationship dynamicRelationship) throws IOException {
+        writeStartElement("dynamicRelationship");
+
+        if (dynamicRelationship != null) {
+            writeTextElement("name", dynamicRelationship.name());
+            writeTextElement("description", dynamicRelationship.description());
+        }
+
+        writeEndElement();
+    }
+
+    @Override
+    protected void writeReadsAttributes(final List<ReadsAttribute> attributes) throws IOException {
+        writeArray("readsAttributes", attributes, this::writeReadsAttribute);
+    }
+
+    private void writeReadsAttribute(final ReadsAttribute attribute) throws IOException {
+        writeStartElement("attribute");
+        writeTextElement("name", attribute.attribute());
+        writeTextElement("description", attribute.description());
+        writeEndElement();
+    }
+
+    @Override
+    protected void writeWritesAttributes(final List<WritesAttribute> attributes) throws IOException {
+        writeArray("writesAttributes", attributes, this::writeWritesAttribute);
+    }
+
+    private void writeWritesAttribute(final WritesAttribute attribute) throws IOException {
+        writeStartElement("attribute");
+        writeTextElement("name", attribute.attribute());
+        writeTextElement("description", attribute.description());
+        writeEndElement();
+    }
+
+    @Override
+    protected void writeFooter(final ConfigurableComponent component) throws IOException {
+        writeEndElement();
+    }
+
+    @Override
+    protected void writeProvidedServices(final Collection<ProvidedServiceAPI> providedServices) throws IOException {
+        writeStartElement("providedServiceAPIs");
+        writeArray("service", providedServices, this::writeProvidedService);
+        writeEndElement();
+    }
+
+    private void writeProvidedService(final ProvidedServiceAPI service) throws IOException {
+        writeTextElement("className",service.getClassName());
+        writeTextElement("groupId",service.getGroupId());
+        writeTextElement("artifactId",service.getArtifactId());
+        writeTextElement("version",service.getVersion());
+    }
+
+    private <T> void writeArray(final String tagName, final Collection<T> values, final ElementWriter<T> writer) throws IOException {
+        writeStartElement(tagName);
+
+        if (values != null) {
+            for (final T value : values) {
+                writer.write(value);
+            }
+        }
+
+        writeEndElement();
+    }
+
+
+    private void writeTextArray(final String outerTagName, final String elementTagName, final Collection<String> values) throws IOException {
+        writeTextArray(outerTagName, elementTagName, values, String::toString);
+    }
+
+    private <T> void writeTextArray(final String outerTagName, final String elementTagName, final Collection<T> values, final Function<T, String> transform) throws IOException {
+        writeStartElement(outerTagName);
+
+        if (values != null) {
+            for (final T value : values) {
+                writeStartElement(elementTagName);
+                if (value != null) {
+                    writeText(transform.apply(value));
+                }
+                writeEndElement();
+            }
+        }
+
+        writeEndElement();
+    }
+
+    private void writeText(final String text) throws IOException {
+        if (text == null) {
+            return;
+        }
+
+        try {
+            writer.writeCharacters(text);
+        } catch (XMLStreamException e) {
+            throw new IOException(e);
+        }
+    }
+
+    private void writeEmptyElement(final String elementName) throws IOException {
+        try {
+            writer.writeEmptyElement(elementName);
+        } catch (final XMLStreamException e) {
+            throw new IOException(e);
+        }
+    }
+
+    private void writeStartElement(final String elementName) throws IOException {
+        try {
+            writer.writeStartElement(elementName);
+        } catch (final XMLStreamException e) {
+            throw new IOException(e);
+        }
+    }
+
+    private void writeEndElement() throws IOException {
+        try {
+            writer.writeEndElement();;
+        } catch (final XMLStreamException e) {
+            throw new IOException(e);
+        }
+    }
+
+    private void writeTextElement(final String name, final String text) throws IOException {
+        writeStartElement(name);
+        writeText(text);
+        writeEndElement();
+    }
+
+    private void writeBooleanElement(final String name, final boolean value) throws IOException {
+        writeTextElement(name, String.valueOf(value));
+    }
+
+    private interface ElementWriter<T> {
+        void write(T value) throws IOException;
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/1ac5b931/nifi-nar-bundles/nifi-amqp-bundle/nifi-amqp-processors/src/main/java/org/apache/nifi/amqp/processors/AbstractAMQPProcessor.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-amqp-bundle/nifi-amqp-processors/src/main/java/org/apache/nifi/amqp/processors/AbstractAMQPProcessor.java b/nifi-nar-bundles/nifi-amqp-bundle/nifi-amqp-processors/src/main/java/org/apache/nifi/amqp/processors/AbstractAMQPProcessor.java
index 2a83497..238b1be 100644
--- a/nifi-nar-bundles/nifi-amqp-bundle/nifi-amqp-processors/src/main/java/org/apache/nifi/amqp/processors/AbstractAMQPProcessor.java
+++ b/nifi-nar-bundles/nifi-amqp-bundle/nifi-amqp-processors/src/main/java/org/apache/nifi/amqp/processors/AbstractAMQPProcessor.java
@@ -16,17 +16,11 @@
  */
 package org.apache.nifi.amqp.processors;
 
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-import java.util.concurrent.BlockingQueue;
-import java.util.concurrent.LinkedBlockingQueue;
-
-import javax.net.ssl.SSLContext;
-
+import com.rabbitmq.client.Connection;
+import com.rabbitmq.client.ConnectionFactory;
+import com.rabbitmq.client.DefaultSaslConfig;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.nifi.annotation.lifecycle.OnStopped;
-import org.apache.nifi.authentication.exception.ProviderCreationException;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.processor.AbstractProcessor;
@@ -37,9 +31,12 @@ import org.apache.nifi.processor.util.StandardValidators;
 import org.apache.nifi.security.util.SslContextFactory;
 import org.apache.nifi.ssl.SSLContextService;
 
-import com.rabbitmq.client.Connection;
-import com.rabbitmq.client.ConnectionFactory;
-import com.rabbitmq.client.DefaultSaslConfig;
+import javax.net.ssl.SSLContext;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.LinkedBlockingQueue;
 
 
 /**
@@ -224,7 +221,7 @@ abstract class AbstractAMQPProcessor<T extends AMQPWorker> extends AbstractProce
         final SSLContextService sslService = context.getProperty(SSL_CONTEXT_SERVICE).asControllerService(SSLContextService.class);
         // if the property to use cert authentication is set but the SSL service hasn't been configured, throw an exception.
         if (useCertAuthentication && sslService == null) {
-            throw new ProviderCreationException("This processor is configured to use cert authentication, " +
+            throw new IllegalStateException("This processor is configured to use cert authentication, " +
                     "but the SSL Context Service hasn't been configured. You need to configure the SSL Context Service.");
         }
         final String rawClientAuth = context.getProperty(CLIENT_AUTH).getValue();
@@ -237,7 +234,7 @@ abstract class AbstractAMQPProcessor<T extends AMQPWorker> extends AbstractProce
                 try {
                     clientAuth = SSLContextService.ClientAuth.valueOf(rawClientAuth);
                 } catch (final IllegalArgumentException iae) {
-                    throw new ProviderCreationException(String.format("Unrecognized client auth '%s'. Possible values are [%s]",
+                    throw new IllegalStateException(String.format("Unrecognized client auth '%s'. Possible values are [%s]",
                             rawClientAuth, StringUtils.join(SslContextFactory.ClientAuth.values(), ", ")));
                 }
             }

http://git-wip-us.apache.org/repos/asf/nifi/blob/1ac5b931/nifi-nar-bundles/nifi-amqp-bundle/nifi-amqp-processors/src/test/java/org/apache/nifi/amqp/processors/AbstractAMQPProcessorTest.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-amqp-bundle/nifi-amqp-processors/src/test/java/org/apache/nifi/amqp/processors/AbstractAMQPProcessorTest.java b/nifi-nar-bundles/nifi-amqp-bundle/nifi-amqp-processors/src/test/java/org/apache/nifi/amqp/processors/AbstractAMQPProcessorTest.java
index bc4c32d..8cbb8a3 100644
--- a/nifi-nar-bundles/nifi-amqp-bundle/nifi-amqp-processors/src/test/java/org/apache/nifi/amqp/processors/AbstractAMQPProcessorTest.java
+++ b/nifi-nar-bundles/nifi-amqp-bundle/nifi-amqp-processors/src/test/java/org/apache/nifi/amqp/processors/AbstractAMQPProcessorTest.java
@@ -16,10 +16,7 @@
  */
 package org.apache.nifi.amqp.processors;
 
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
-
-import org.apache.nifi.authentication.exception.ProviderCreationException;
+import com.rabbitmq.client.Connection;
 import org.apache.nifi.processor.ProcessContext;
 import org.apache.nifi.processor.ProcessSession;
 import org.apache.nifi.processor.exception.ProcessException;
@@ -29,7 +26,8 @@ import org.apache.nifi.util.TestRunners;
 import org.junit.Before;
 import org.junit.Test;
 
-import com.rabbitmq.client.Connection;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
 
 
 /**
@@ -46,7 +44,7 @@ public class AbstractAMQPProcessorTest {
         testRunner = TestRunners.newTestRunner(processor);
     }
 
-    @Test(expected = ProviderCreationException.class)
+    @Test(expected = IllegalStateException.class)
     public void testConnectToCassandraWithSSLBadClientAuth() throws Exception {
         SSLContextService sslService = mock(SSLContextService.class);
         when(sslService.getIdentifier()).thenReturn("ssl-context");
@@ -63,7 +61,7 @@ public class AbstractAMQPProcessorTest {
         processor.onTrigger(testRunner.getProcessContext(), testRunner.getProcessSessionFactory());
     }
 
-    @Test(expected = ProviderCreationException.class)
+    @Test(expected = IllegalStateException.class)
     public void testInvalidSSLConfiguration() throws Exception {
         // it's invalid to have use_cert_auth enabled and not have the SSL Context Service configured
         testRunner.setProperty(AbstractAMQPProcessor.USE_CERT_AUTHENTICATION, "true");

http://git-wip-us.apache.org/repos/asf/nifi/blob/1ac5b931/nifi-nar-bundles/nifi-druid-bundle/nifi-druid-controller-service-api/pom.xml
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-druid-bundle/nifi-druid-controller-service-api/pom.xml b/nifi-nar-bundles/nifi-druid-bundle/nifi-druid-controller-service-api/pom.xml
index f49fd9e..f26451c 100644
--- a/nifi-nar-bundles/nifi-druid-bundle/nifi-druid-controller-service-api/pom.xml
+++ b/nifi-nar-bundles/nifi-druid-bundle/nifi-druid-controller-service-api/pom.xml
@@ -28,7 +28,6 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-api</artifactId>
-            <scope>provided</scope>
         </dependency>
         <!-- Forcing to 3.2.2 to avoid a CVE in 3.2.1 .. coming in transitively from tranquility core-->
         <dependency>

http://git-wip-us.apache.org/repos/asf/nifi/blob/1ac5b931/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-client-service-api/pom.xml
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-client-service-api/pom.xml b/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-client-service-api/pom.xml
index f1f1ff0..736a4f3 100644
--- a/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-client-service-api/pom.xml
+++ b/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-client-service-api/pom.xml
@@ -36,7 +36,6 @@
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-utils</artifactId>
             <version>1.9.0-SNAPSHOT</version>
-            <scope>provided</scope>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>

http://git-wip-us.apache.org/repos/asf/nifi/blob/1ac5b931/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-client-service/pom.xml
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-client-service/pom.xml b/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-client-service/pom.xml
index fbb938c..0467763 100644
--- a/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-client-service/pom.xml
+++ b/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-client-service/pom.xml
@@ -41,7 +41,6 @@
         <dependency>
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-utils</artifactId>
-            <scope>provided</scope>
             <version>1.9.0-SNAPSHOT</version>
         </dependency>
         <dependency>

http://git-wip-us.apache.org/repos/asf/nifi/blob/1ac5b931/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/DocGenerator.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/DocGenerator.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/DocGenerator.java
index 4a01b35..aa2dbe0 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/DocGenerator.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/DocGenerator.java
@@ -142,7 +142,7 @@ public class DocGenerator {
      * that specific type of class
      */
     private static DocumentationWriter getDocumentWriter(final ExtensionManager extensionManager,
-                                                         final Class<? extends ConfigurableComponent> componentClass) {
+                                                                  final Class<? extends ConfigurableComponent> componentClass) {
         if (Processor.class.isAssignableFrom(componentClass)) {
             return new HtmlProcessorDocumentationWriter(extensionManager);
         } else if (ControllerService.class.isAssignableFrom(componentClass)) {

http://git-wip-us.apache.org/repos/asf/nifi/blob/1ac5b931/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/java/org/apache/nifi/documentation/html/HtmlDocumentationWriterTest.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/java/org/apache/nifi/documentation/html/HtmlDocumentationWriterTest.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/java/org/apache/nifi/documentation/html/HtmlDocumentationWriterTest.java
index 585fe1e..c37ebeb 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/java/org/apache/nifi/documentation/html/HtmlDocumentationWriterTest.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/java/org/apache/nifi/documentation/html/HtmlDocumentationWriterTest.java
@@ -16,8 +16,8 @@
  */
 package org.apache.nifi.documentation.html;
 
-import org.apache.nifi.annotation.behavior.SystemResourceConsideration;
 import org.apache.nifi.annotation.behavior.SystemResource;
+import org.apache.nifi.annotation.behavior.SystemResourceConsideration;
 import org.apache.nifi.controller.ControllerService;
 import org.apache.nifi.documentation.DocumentationWriter;
 import org.apache.nifi.documentation.example.ControllerServiceWithLogger;

http://git-wip-us.apache.org/repos/asf/nifi/blob/1ac5b931/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/java/org/apache/nifi/documentation/html/ProcessorDocumentationWriterTest.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/java/org/apache/nifi/documentation/html/ProcessorDocumentationWriterTest.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/java/org/apache/nifi/documentation/html/ProcessorDocumentationWriterTest.java
index 06c8e83..573cd0c 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/java/org/apache/nifi/documentation/html/ProcessorDocumentationWriterTest.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/java/org/apache/nifi/documentation/html/ProcessorDocumentationWriterTest.java
@@ -16,12 +16,6 @@
  */
 package org.apache.nifi.documentation.html;
 
-import static org.apache.nifi.documentation.html.XmlValidator.assertContains;
-import static org.apache.nifi.documentation.html.XmlValidator.assertNotContains;
-
-import java.io.ByteArrayOutputStream;
-import java.io.IOException;
-
 import org.apache.nifi.annotation.behavior.SystemResource;
 import org.apache.nifi.annotation.behavior.SystemResourceConsideration;
 import org.apache.nifi.annotation.documentation.CapabilityDescription;
@@ -37,6 +31,12 @@ import org.apache.nifi.nar.StandardExtensionDiscoveringManager;
 import org.junit.Assert;
 import org.junit.Test;
 
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+
+import static org.apache.nifi.documentation.html.XmlValidator.assertContains;
+import static org.apache.nifi.documentation.html.XmlValidator.assertNotContains;
+
 public class ProcessorDocumentationWriterTest {
 
     @Test

http://git-wip-us.apache.org/repos/asf/nifi/blob/1ac5b931/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-jetty/src/main/java/org/apache/nifi/web/server/JettyServer.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-jetty/src/main/java/org/apache/nifi/web/server/JettyServer.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-jetty/src/main/java/org/apache/nifi/web/server/JettyServer.java
index 401d8e9..5261f76 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-jetty/src/main/java/org/apache/nifi/web/server/JettyServer.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-jetty/src/main/java/org/apache/nifi/web/server/JettyServer.java
@@ -1138,8 +1138,11 @@ public class JettyServer implements NiFiServer, ExtensionUiLoader {
         } catch (Exception ex) {
             logger.warn("Failed to stop web server", ex);
         }
+
         try {
-            narAutoLoader.stop();
+            if (narAutoLoader != null) {
+                narAutoLoader.stop();
+            }
         } catch (Exception e) {
             logger.warn("Failed to stop NAR auto-loader", e);
         }

http://git-wip-us.apache.org/repos/asf/nifi/blob/1ac5b931/nifi-nar-bundles/nifi-groovyx-bundle/nifi-groovyx-processors/pom.xml
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-groovyx-bundle/nifi-groovyx-processors/pom.xml b/nifi-nar-bundles/nifi-groovyx-bundle/nifi-groovyx-processors/pom.xml
index cc0601b..23202e1 100644
--- a/nifi-nar-bundles/nifi-groovyx-bundle/nifi-groovyx-processors/pom.xml
+++ b/nifi-nar-bundles/nifi-groovyx-bundle/nifi-groovyx-processors/pom.xml
@@ -35,7 +35,6 @@
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-processor-utils</artifactId>
             <version>1.9.0-SNAPSHOT</version>
-            <scope>provided</scope>
         </dependency>
         <dependency>
             <groupId>org.codehaus.groovy</groupId>

http://git-wip-us.apache.org/repos/asf/nifi/blob/1ac5b931/nifi-nar-bundles/nifi-mongodb-bundle/nifi-mongodb-processors/src/main/java/org/apache/nifi/processors/mongodb/AbstractMongoProcessor.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-mongodb-bundle/nifi-mongodb-processors/src/main/java/org/apache/nifi/processors/mongodb/AbstractMongoProcessor.java b/nifi-nar-bundles/nifi-mongodb-bundle/nifi-mongodb-processors/src/main/java/org/apache/nifi/processors/mongodb/AbstractMongoProcessor.java
index 22f25b5..69de94b 100644
--- a/nifi-nar-bundles/nifi-mongodb-bundle/nifi-mongodb-processors/src/main/java/org/apache/nifi/processors/mongodb/AbstractMongoProcessor.java
+++ b/nifi-nar-bundles/nifi-mongodb-bundle/nifi-mongodb-processors/src/main/java/org/apache/nifi/processors/mongodb/AbstractMongoProcessor.java
@@ -29,7 +29,6 @@ import com.mongodb.client.MongoDatabase;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.nifi.annotation.lifecycle.OnScheduled;
 import org.apache.nifi.annotation.lifecycle.OnStopped;
-import org.apache.nifi.authentication.exception.ProviderCreationException;
 import org.apache.nifi.components.AllowableValue;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.ValidationContext;
@@ -254,7 +253,7 @@ public abstract class AbstractMongoProcessor extends AbstractProcessor {
                 try {
                     clientAuth = SSLContextService.ClientAuth.valueOf(rawClientAuth);
                 } catch (final IllegalArgumentException iae) {
-                    throw new ProviderCreationException(String.format("Unrecognized client auth '%s'. Possible values are [%s]",
+                    throw new IllegalStateException(String.format("Unrecognized client auth '%s'. Possible values are [%s]",
                             rawClientAuth, StringUtils.join(SslContextFactory.ClientAuth.values(), ", ")));
                 }
             }

http://git-wip-us.apache.org/repos/asf/nifi/blob/1ac5b931/nifi-nar-bundles/nifi-mongodb-bundle/nifi-mongodb-processors/src/test/java/org/apache/nifi/processors/mongodb/AbstractMongoProcessorTest.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-mongodb-bundle/nifi-mongodb-processors/src/test/java/org/apache/nifi/processors/mongodb/AbstractMongoProcessorTest.java b/nifi-nar-bundles/nifi-mongodb-bundle/nifi-mongodb-processors/src/test/java/org/apache/nifi/processors/mongodb/AbstractMongoProcessorTest.java
index 9951fdf..1ae2b82 100644
--- a/nifi-nar-bundles/nifi-mongodb-bundle/nifi-mongodb-processors/src/test/java/org/apache/nifi/processors/mongodb/AbstractMongoProcessorTest.java
+++ b/nifi-nar-bundles/nifi-mongodb-bundle/nifi-mongodb-processors/src/test/java/org/apache/nifi/processors/mongodb/AbstractMongoProcessorTest.java
@@ -16,14 +16,8 @@
  */
 package org.apache.nifi.processors.mongodb;
 
-import static org.junit.Assert.assertNotNull;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
-import static org.mockito.Mockito.any;
-
-import javax.net.ssl.SSLContext;
-
-import org.apache.nifi.authentication.exception.ProviderCreationException;
+import com.mongodb.MongoClientOptions;
+import com.mongodb.MongoClientOptions.Builder;
 import org.apache.nifi.processor.ProcessContext;
 import org.apache.nifi.processor.ProcessSession;
 import org.apache.nifi.processor.exception.ProcessException;
@@ -34,8 +28,12 @@ import org.apache.nifi.util.TestRunners;
 import org.junit.Before;
 import org.junit.Test;
 
-import com.mongodb.MongoClientOptions;
-import com.mongodb.MongoClientOptions.Builder;
+import javax.net.ssl.SSLContext;
+
+import static org.junit.Assert.assertNotNull;
+import static org.mockito.Mockito.any;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
 
 public class AbstractMongoProcessorTest {
 
@@ -67,7 +65,7 @@ public class AbstractMongoProcessorTest {
         assertNotNull(processor.mongoClient);
     }
 
-    @Test(expected = ProviderCreationException.class)
+    @Test(expected = IllegalStateException.class)
     public void testcreateClientWithSSLBadClientAuth() throws Exception {
         SSLContextService sslService = mock(SSLContextService.class);
         SSLContext sslContext = mock(SSLContext.class);

http://git-wip-us.apache.org/repos/asf/nifi/blob/1ac5b931/nifi-nar-bundles/nifi-mongodb-bundle/nifi-mongodb-services/src/main/java/org/apache/nifi/mongodb/MongoDBControllerService.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-mongodb-bundle/nifi-mongodb-services/src/main/java/org/apache/nifi/mongodb/MongoDBControllerService.java b/nifi-nar-bundles/nifi-mongodb-bundle/nifi-mongodb-services/src/main/java/org/apache/nifi/mongodb/MongoDBControllerService.java
index a851ad6..c731fe4 100644
--- a/nifi-nar-bundles/nifi-mongodb-bundle/nifi-mongodb-services/src/main/java/org/apache/nifi/mongodb/MongoDBControllerService.java
+++ b/nifi-nar-bundles/nifi-mongodb-bundle/nifi-mongodb-services/src/main/java/org/apache/nifi/mongodb/MongoDBControllerService.java
@@ -28,7 +28,6 @@ import org.apache.nifi.annotation.documentation.Tags;
 import org.apache.nifi.annotation.lifecycle.OnDisabled;
 import org.apache.nifi.annotation.lifecycle.OnEnabled;
 import org.apache.nifi.annotation.lifecycle.OnStopped;
-import org.apache.nifi.authentication.exception.ProviderCreationException;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.controller.AbstractControllerService;
 import org.apache.nifi.controller.ConfigurationContext;
@@ -83,7 +82,7 @@ public class MongoDBControllerService extends AbstractControllerService implemen
                 try {
                     clientAuth = SSLContextService.ClientAuth.valueOf(rawClientAuth);
                 } catch (final IllegalArgumentException iae) {
-                    throw new ProviderCreationException(String.format("Unrecognized client auth '%s'. Possible values are [%s]",
+                    throw new IllegalStateException(String.format("Unrecognized client auth '%s'. Possible values are [%s]",
                             rawClientAuth, StringUtils.join(SslContextFactory.ClientAuth.values(), ", ")));
                 }
             }

http://git-wip-us.apache.org/repos/asf/nifi/blob/1ac5b931/nifi-nar-bundles/nifi-standard-services/nifi-hbase-client-service-api/pom.xml
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-hbase-client-service-api/pom.xml b/nifi-nar-bundles/nifi-standard-services/nifi-hbase-client-service-api/pom.xml
index e50db19..51fd212 100644
--- a/nifi-nar-bundles/nifi-standard-services/nifi-hbase-client-service-api/pom.xml
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-hbase-client-service-api/pom.xml
@@ -31,10 +31,5 @@
             <artifactId>nifi-api</artifactId>
             <scope>provided</scope>
         </dependency>
-        <dependency>
-            <groupId>org.apache.nifi</groupId>
-            <artifactId>nifi-utils</artifactId>
-            <version>1.9.0-SNAPSHOT</version>
-        </dependency>
     </dependencies>
 </project>