From d7cf25be9a157f6934593a325f906a0fc85e4007 Mon Sep 17 00:00:00 2001 From: danbress Date: Sat, 31 Jan 2015 09:02:23 -0500 Subject: [PATCH 01/26] NIFI-280 - Adding new documentation framework component and incorporating in framework --- .../nifi-framework/nifi-documentation/pom.xml | 49 +++ .../ConfigurableComponentInitializer.java | 38 ++ .../nifi/documentation/DocGenerator.java | 182 ++++++++ .../documentation/DocumentationWriter.java | 33 ++ .../html/HtmlDocumentationWriter.java | 415 ++++++++++++++++++ .../HtmlProcessorDocumentationWriter.java | 60 +++ .../init/ControllerServiceInitializer.java | 38 ++ .../init/ProcessorInitializer.java | 37 ++ .../init/ReportingTaskingInitializer.java | 37 ++ ...ontrollerServiceInitializationContext.java | 40 ++ .../mock/MockControllerServiceLookup.java | 55 +++ .../MockProcessorInitializationContext.java | 45 ++ .../MockReportingInitializationContext.java | 63 +++ .../src/main/resources/apache.license | 12 + .../FullyDocumentedControllerService.java | 57 +++ .../example/FullyDocumentedProcessor.java | 100 +++++ .../example/FullyDocumentedReportingTask.java | 50 +++ .../html/HtmlDocumentationWriterTest.java | 56 +++ .../ProcessorDocumentationWriterTest.java | 69 +++ .../nifi/documentation/html/XmlValidator.java | 50 +++ .../nifi-framework/nifi-runtime/pom.xml | 5 + .../src/main/java/org/apache/nifi/NiFi.java | 3 + .../nifi-framework/pom.xml | 1 + .../nifi-framework-bundle/pom.xml | 5 + 24 files changed, 1500 insertions(+) create mode 100644 nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/pom.xml create mode 100644 nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/ConfigurableComponentInitializer.java create mode 100644 nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/DocGenerator.java create mode 100644 nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/DocumentationWriter.java create mode 100644 nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/html/HtmlDocumentationWriter.java create mode 100644 nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/html/HtmlProcessorDocumentationWriter.java create mode 100644 nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/init/ControllerServiceInitializer.java create mode 100644 nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/init/ProcessorInitializer.java create mode 100644 nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/init/ReportingTaskingInitializer.java create mode 100644 nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/mock/MockControllerServiceInitializationContext.java create mode 100644 nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/mock/MockControllerServiceLookup.java create mode 100644 nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/mock/MockProcessorInitializationContext.java create mode 100644 nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/mock/MockReportingInitializationContext.java create mode 100644 nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/resources/apache.license create mode 100644 nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/java/org/apache/nifi/documentation/example/FullyDocumentedControllerService.java create mode 100644 nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/java/org/apache/nifi/documentation/example/FullyDocumentedProcessor.java create mode 100644 nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/java/org/apache/nifi/documentation/example/FullyDocumentedReportingTask.java create mode 100644 nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/java/org/apache/nifi/documentation/html/HtmlDocumentationWriterTest.java create mode 100644 nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/java/org/apache/nifi/documentation/html/ProcessorDocumentationWriterTest.java create mode 100644 nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/java/org/apache/nifi/documentation/html/XmlValidator.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/pom.xml b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/pom.xml new file mode 100644 index 0000000000..f35c8cd83c --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/pom.xml @@ -0,0 +1,49 @@ + + + + 4.0.0 + + org.apache.nifi + nifi-framework + 0.0.2-incubating-SNAPSHOT + + nifi-documentation + + + org.apache.nifi + nifi-nar-utils + + + org.apache.nifi + nifi-api + + + org.apache.nifi + nifi-properties + + + commons-io + commons-io + + + org.apache.commons + commons-lang3 + + + org.apache.nifi + nifi-processor-utils + test + + + diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/ConfigurableComponentInitializer.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/ConfigurableComponentInitializer.java new file mode 100644 index 0000000000..164d212bd0 --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/ConfigurableComponentInitializer.java @@ -0,0 +1,38 @@ +/* + * 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 org.apache.nifi.reporting.InitializationException; + +/** + * An interface for initializing 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; +} diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/DocGenerator.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/DocGenerator.java new file mode 100644 index 0000000000..9a373565db --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/DocGenerator.java @@ -0,0 +1,182 @@ +/* + * 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 java.io.BufferedOutputStream; +import java.io.File; +import java.io.FileOutputStream; +import java.io.FilenameFilter; +import java.io.IOException; +import java.io.OutputStream; +import java.util.HashSet; +import java.util.Set; + +import org.apache.nifi.components.ConfigurableComponent; +import org.apache.nifi.controller.ControllerService; +import org.apache.nifi.documentation.html.HtmlDocumentationWriter; +import org.apache.nifi.documentation.html.HtmlProcessorDocumentationWriter; +import org.apache.nifi.documentation.init.ControllerServiceInitializer; +import org.apache.nifi.documentation.init.ProcessorInitializer; +import org.apache.nifi.documentation.init.ReportingTaskingInitializer; +import org.apache.nifi.nar.ExtensionManager; +import org.apache.nifi.processor.Processor; +import org.apache.nifi.reporting.InitializationException; +import org.apache.nifi.reporting.ReportingTask; +import org.apache.nifi.util.NiFiProperties; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Uses the ExtensionManager to get a list of Processor, ControllerService, and + * Reporting Task classes that were loaded and generate documentation for them. + * + * + */ +public class DocGenerator { + + private static final Logger logger = LoggerFactory.getLogger(DocGenerator.class); + + /** + * Generates documentation into the work/docs dir specified by + * NiFiProperties. + * + * @param properties + */ + public static void generate(final NiFiProperties properties) { + @SuppressWarnings("rawtypes") + final Set extensionClasses = new HashSet<>(); + extensionClasses.addAll(ExtensionManager.getExtensions(Processor.class)); + extensionClasses.addAll(ExtensionManager.getExtensions(ControllerService.class)); + extensionClasses.addAll(ExtensionManager.getExtensions(ReportingTask.class)); + + final File explodedNiFiDocsDir = properties.getComponentDocumentationWorkingDirectory(); + + logger.debug("Generating documentation for: " + extensionClasses.size() + " components in: " + + explodedNiFiDocsDir); + + for (final Class extensionClass : extensionClasses) { + if (ConfigurableComponent.class.isAssignableFrom(extensionClass)) { + final Class componentClass = extensionClass + .asSubclass(ConfigurableComponent.class); + try { + logger.debug("Documenting: " + componentClass); + document(explodedNiFiDocsDir, componentClass); + } catch (Exception e) { + logger.warn("Unable to document: " + componentClass); + } + } + } + } + + /** + * Generates the documentation for a particular configurable comopnent. Will + * check to see if an "additionalDetails.html" file exists and will link + * that from the generated documentation. + * + * @param docsDir + * the work\docs\components dir to stick component documentation + * in + * @param componentClass + * the class to document + * @throws InstantiationException + * @throws IllegalAccessException + * @throws IOException + * @throws InitializationException + */ + private static void document(final File docsDir, final Class componentClass) + throws InstantiationException, IllegalAccessException, IOException, InitializationException { + + final ConfigurableComponent component = componentClass.newInstance(); + final ConfigurableComponentInitializer initializer = getComponentInitializer(componentClass); + initializer.initialize(component); + + final DocumentationWriter writer = getDocumentWriter(componentClass); + + final File directory = new File(docsDir, componentClass.getCanonicalName()); + directory.mkdirs(); + + final File baseDocumenationFile = new File(directory, "index.html"); + if (baseDocumenationFile.exists()) { + logger.warn(baseDocumenationFile + " already exists! Overwriting!"); + } + + try (final OutputStream output = new BufferedOutputStream(new FileOutputStream(baseDocumenationFile))) { + writer.write(component, output, hasAdditionalInfo(directory)); + } + } + + /** + * Returns the DocumentationWriter for the type of component. Currently + * Processor, ControllerService, and ReportingTask are supported. + * + * @param componentClass + * the class that requires a DocumentationWriter + * @return a DocumentationWriter capable of generating documentation for + * that specific type of class + */ + private static DocumentationWriter getDocumentWriter(final Class componentClass) { + if (Processor.class.isAssignableFrom(componentClass)) { + return new HtmlProcessorDocumentationWriter(); + } else if (ControllerService.class.isAssignableFrom(componentClass)) { + return new HtmlDocumentationWriter(); + } else if (ReportingTask.class.isAssignableFrom(componentClass)) { + return new HtmlDocumentationWriter(); + } + + return null; + } + + /** + * 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 + */ + private static ConfigurableComponentInitializer getComponentInitializer( + final Class 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; + } + + /** + * Checks to see if a directory to write to has an additionalDetails.html in + * it already. + * + * @param directory + * @return true if additionalDetails.html exists, false otherwise. + */ + private static boolean hasAdditionalInfo(File directory) { + return directory.list(new FilenameFilter() { + + @Override + public boolean accept(File dir, String name) { + return name.equalsIgnoreCase(HtmlDocumentationWriter.ADDITIONAL_DETAILS_HTML); + } + + }).length > 0; + } +} diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/DocumentationWriter.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/DocumentationWriter.java new file mode 100644 index 0000000000..5a3c5d8a80 --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/DocumentationWriter.java @@ -0,0 +1,33 @@ +/* + * 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 java.io.IOException; +import java.io.OutputStream; + +import org.apache.nifi.components.ConfigurableComponent; + +/** + * Generates documentation for an instance of a ConfigurableComponent + * + * + */ +public interface DocumentationWriter { + + void write(ConfigurableComponent configurableComponent, OutputStream streamToWriteTo, + boolean includesAdditionalDocumentation) throws IOException; +} diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/html/HtmlDocumentationWriter.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/html/HtmlDocumentationWriter.java new file mode 100644 index 0000000000..bf9bd131b0 --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/html/HtmlDocumentationWriter.java @@ -0,0 +1,415 @@ +/* + * 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.html; + +import java.io.IOException; +import java.io.OutputStream; +import java.util.List; + +import javax.xml.stream.FactoryConfigurationError; +import javax.xml.stream.XMLOutputFactory; +import javax.xml.stream.XMLStreamException; +import javax.xml.stream.XMLStreamWriter; + +import org.apache.commons.io.IOUtils; +import org.apache.commons.lang3.StringUtils; +import org.apache.nifi.components.AllowableValue; +import org.apache.nifi.components.ConfigurableComponent; +import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.documentation.DocumentationWriter; +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.documentation.Tags; + +/** + * Generates HTML documentation for a ConfigurableComponent. This class is used + * to generate documentation for ControllerService and ReportingTask because + * they have no additional information. + * + * + */ +public class HtmlDocumentationWriter implements DocumentationWriter { + + /** + * The filename where additional user specified information may be stored. + */ + public static final String ADDITIONAL_DETAILS_HTML = "additionalDetails.html"; + + /** + * The apache licence to apply to the top of the generated html + */ + private static final String apacheLicense; + + static { + String value = null; + try { + value = IOUtils.toString(ClassLoader.getSystemResourceAsStream("apache.license")); + } catch (IOException e) { + e.printStackTrace(); + } + + apacheLicense = value; + } + + @Override + public void write(final ConfigurableComponent configurableComponent, final OutputStream streamToWriteTo, + final boolean includesAdditionalDocumentation) throws IOException { + + try { + XMLStreamWriter xmlStreamWriter = XMLOutputFactory.newInstance().createXMLStreamWriter(streamToWriteTo, + "UTF-8"); + xmlStreamWriter.writeDTD(""); + xmlStreamWriter.writeStartElement("html"); + xmlStreamWriter.writeAttribute("lang", "en"); + writeHead(configurableComponent, xmlStreamWriter); + writeBody(configurableComponent, xmlStreamWriter, includesAdditionalDocumentation); + xmlStreamWriter.writeEndElement(); + xmlStreamWriter.close(); + } catch (XMLStreamException | FactoryConfigurationError e) { + throw new IOException("Unable to create XMLOutputStream", e); + } + } + + /** + * Writes the head portion of the HTML documentation. + * + * @param configurableComponent + * the component to describe + * @param xmlStreamWriter + * the stream to write to + * @throws XMLStreamException + * thrown if there was a problem writing to the stream + */ + protected void writeHead(final ConfigurableComponent configurableComponent, final XMLStreamWriter xmlStreamWriter) + throws XMLStreamException { + + // write the apache license + xmlStreamWriter.writeComment(apacheLicense); + xmlStreamWriter.writeStartElement("head"); + xmlStreamWriter.writeStartElement("meta"); + xmlStreamWriter.writeAttribute("charset", "utf-8"); + xmlStreamWriter.writeEndElement(); + writeSimpleElement(xmlStreamWriter, "title", getTitle(configurableComponent)); + + xmlStreamWriter.writeStartElement("link"); + xmlStreamWriter.writeAttribute("rel", "stylesheet"); + xmlStreamWriter.writeAttribute("href", "../../css/component-usage.css"); + xmlStreamWriter.writeAttribute("type", "text/css"); + xmlStreamWriter.writeEndElement(); + + xmlStreamWriter.writeEndElement(); + } + + /** + * Gets the class name of the component. + * + * @param configurableComponent + * the component to describe + * @return the class name of the component + */ + protected String getTitle(final ConfigurableComponent configurableComponent) { + return configurableComponent.getClass().getSimpleName(); + } + + /** + * Writes the body section of the documentation, this consists of the + * component description, the tags, and the PropertyDescriptors. + * + * @param configurableComponent + * the component to describe + * @param xmlStreamWriter + * the stream writer + * @param hasAdditionalDetails + * whether there are additional details present or not + * @throws XMLStreamException + * thrown if there was a problem writing to the XML stream + */ + private final void writeBody(final ConfigurableComponent configurableComponent, + final XMLStreamWriter xmlStreamWriter, final boolean hasAdditionalDetails) throws XMLStreamException { + xmlStreamWriter.writeStartElement("body"); + writeDescription(configurableComponent, xmlStreamWriter, hasAdditionalDetails); + writeTags(configurableComponent, xmlStreamWriter); + writeProperties(configurableComponent, xmlStreamWriter); + writeAdditionalBodyInfo(configurableComponent, xmlStreamWriter); + xmlStreamWriter.writeEndElement(); + } + + /** + * This method may be overridden by sub classes to write additional + * information to the body of the documentation. + * + * @param configurableComponent + * the component to describe + * @param xmlStreamWriter + * the stream writer + * @throws XMLStreamException + * thrown if there was a problem writing to the XML stream + */ + protected void writeAdditionalBodyInfo(final ConfigurableComponent configurableComponent, + final XMLStreamWriter xmlStreamWriter) throws XMLStreamException { + } + + /** + * Writes the tags attached to a ConfigurableComponent. + * + * @param configurableComponent + * @param xmlStreamWriter + * @throws XMLStreamException + */ + private void writeTags(final ConfigurableComponent configurableComponent, final XMLStreamWriter xmlStreamWriter) + throws XMLStreamException { + final Tags tags = configurableComponent.getClass().getAnnotation(Tags.class); + xmlStreamWriter.writeStartElement("p"); + if (tags != null) { + final String tagString = StringUtils.join(tags.value(), ", "); + xmlStreamWriter.writeCharacters("Tags: "); + xmlStreamWriter.writeCharacters(tagString); + } else { + xmlStreamWriter.writeCharacters("No Tags provided."); + } + xmlStreamWriter.writeEndElement(); + + } + + /** + * Writes a description of the configurable component. + * + * @param configurableComponent + * the component to describe + * @param xmlStreamWriter + * the stream writer + * @param hasAdditionalDetails + * whether there are additional details available as + * 'additionalDetails.html' + * @throws XMLStreamException + * thrown if there was a problem writing to the XML stream + */ + protected void writeDescription(final ConfigurableComponent configurableComponent, + final XMLStreamWriter xmlStreamWriter, final boolean hasAdditionalDetails) throws XMLStreamException { + writeSimpleElement(xmlStreamWriter, "h2", "Description: "); + writeSimpleElement(xmlStreamWriter, "p", getDescription(configurableComponent)); + if (hasAdditionalDetails) { + xmlStreamWriter.writeStartElement("p"); + + writeLink(xmlStreamWriter, "Additional Details...", ADDITIONAL_DETAILS_HTML); + + xmlStreamWriter.writeEndElement(); + } + } + + /** + * Gets a description of the ConfigurableComponent using the + * CapabilityDescription annotation. + * + * @param configurableComponent + * the component to describe + * @return a description of the configurableComponent + */ + protected String getDescription(final ConfigurableComponent configurableComponent) { + final CapabilityDescription capabilityDescription = configurableComponent.getClass().getAnnotation( + CapabilityDescription.class); + + final String description; + if (capabilityDescription != null) { + description = capabilityDescription.value(); + } else { + description = "No description provided."; + } + + return description; + } + + /** + * Writes the PropertyDescriptors out as a table. + * + * @param configurableComponent + * the component to describe + * @param xmlStreamWriter + * the stream writer + * @throws XMLStreamException + * thrown if there was a problem writing to the XML Stream + */ + protected void writeProperties(final ConfigurableComponent configurableComponent, + final XMLStreamWriter xmlStreamWriter) throws XMLStreamException { + xmlStreamWriter.writeStartElement("p"); + writeSimpleElement(xmlStreamWriter, "strong", "Properties: "); + xmlStreamWriter.writeEndElement(); + xmlStreamWriter.writeStartElement("p"); + xmlStreamWriter.writeCharacters("In the list below, the names of required properties appear in "); + writeSimpleElement(xmlStreamWriter, "strong", "bold"); + xmlStreamWriter.writeCharacters(". Any" + + "other properties (not in bold) are considered optional. The table also " + + "indicates any default values, whether a property supports the "); + writeLink(xmlStreamWriter, "NiFi Expression Language (or simply EL)", + "../../html/expression-language-guide.html"); + xmlStreamWriter.writeCharacters(", and whether a property is considered " + + "\"sensitive\", meaning that its value will be encrypted. Before entering a " + + "value in a sensitive property, ensure that the "); + writeSimpleElement(xmlStreamWriter, "strong", "nifi.properties"); + xmlStreamWriter.writeCharacters(" file has " + "an entry for the property "); + writeSimpleElement(xmlStreamWriter, "strong", "nifi.sensitive.props.key"); + xmlStreamWriter.writeCharacters("."); + xmlStreamWriter.writeEndElement(); + + boolean containsSensitiveElement = false; + + List properties = configurableComponent.getPropertyDescriptors(); + if (properties.size() > 0) { + xmlStreamWriter.writeStartElement("table"); + + // write the header row + xmlStreamWriter.writeStartElement("tr"); + writeSimpleElement(xmlStreamWriter, "th", "Name"); + writeSimpleElement(xmlStreamWriter, "th", "Description"); + writeSimpleElement(xmlStreamWriter, "th", "Default Value"); + writeSimpleElement(xmlStreamWriter, "th", "Valid Values"); + xmlStreamWriter.writeStartElement("th"); + writeLink(xmlStreamWriter, "EL", "../../html/expression-language-guide.html"); + xmlStreamWriter.writeEndElement(); + xmlStreamWriter.writeEndElement(); + + // write the individual properties + for (PropertyDescriptor property : properties) { + containsSensitiveElement |= property.isSensitive(); + xmlStreamWriter.writeStartElement("tr"); + xmlStreamWriter.writeStartElement("td"); + if (property.isRequired()) { + writeSimpleElement(xmlStreamWriter, "strong", property.getDisplayName()); + } else { + xmlStreamWriter.writeCharacters(property.getDisplayName()); + } + if (property.isSensitive()) { + writeSensitiveImg(xmlStreamWriter); + } + xmlStreamWriter.writeEndElement(); + writeSimpleElement(xmlStreamWriter, "td", property.getDescription()); + writeSimpleElement(xmlStreamWriter, "td", property.getDefaultValue()); + writeValidValues(xmlStreamWriter, property); + writeSimpleElement(xmlStreamWriter, "td", property.isExpressionLanguageSupported() ? "Yes" : "No"); + xmlStreamWriter.writeEndElement(); + } + + // TODO support dynamic properties... + xmlStreamWriter.writeEndElement(); + + if (containsSensitiveElement) { + writeSensitiveImg(xmlStreamWriter); + xmlStreamWriter.writeCharacters(" indicates that a property is a sensitive property"); + } + + } else { + writeSimpleElement(xmlStreamWriter, "p", "This component has no required or optional properties."); + } + } + + private void writeSensitiveImg(final XMLStreamWriter xmlStreamWriter) throws XMLStreamException { + xmlStreamWriter.writeCharacters(" "); + xmlStreamWriter.writeStartElement("img"); + xmlStreamWriter.writeAttribute("src", "../../html/images/iconSecure.png"); + xmlStreamWriter.writeAttribute("alt", "Sensitive Property"); + xmlStreamWriter.writeEndElement(); + } + + /** + * Interrogates a PropertyDescriptor to get a list of AllowableValues, if + * there are none, nothing is written to the stream. + * + * @param xmlStreamWriter + * the stream writer to use + * @param property + * the property to describe + * @throws XMLStreamException + * thrown if there was a problem writing to the XML Stream + */ + protected void writeValidValues(XMLStreamWriter xmlStreamWriter, PropertyDescriptor property) + throws XMLStreamException { + xmlStreamWriter.writeStartElement("td"); + if (property.getAllowableValues() != null && property.getAllowableValues().size() > 0) { + xmlStreamWriter.writeStartElement("ul"); + for (AllowableValue value : property.getAllowableValues()) { + writeSimpleElement(xmlStreamWriter, "li", value.getDisplayName()); + } + xmlStreamWriter.writeEndElement(); + } + xmlStreamWriter.writeEndElement(); + } + + /** + * Writes a begin element, then text, then end element for the element of a + * users choosing. Example: <p>text</p> + * + * @param writer + * the stream writer to use + * @param elementName + * the name of the element + * @param characters + * the characters to insert into the element + * @param strong + * whether the characters should be strong or not. + * @throws XMLStreamException + * thrown if there was a problem writing to the stream. + */ + protected final static void writeSimpleElement(final XMLStreamWriter writer, final String elementName, + final String characters, boolean strong) throws XMLStreamException { + writer.writeStartElement(elementName); + if (strong) { + writer.writeStartElement("strong"); + } + writer.writeCharacters(characters); + if (strong) { + writer.writeEndElement(); + } + writer.writeEndElement(); + } + + /** + * Writes a begin element, then text, then end element for the element of a + * users choosing. Example: <p>text</p> + * + * @param writer + * the stream writer to use + * @param elementName + * the name of the element + * @param characters + * the characters to insert into the element + * @throws XMLStreamException + * thrown if there was a problem writing to the stream + */ + protected final static void writeSimpleElement(final XMLStreamWriter writer, final String elementName, + final String characters) throws XMLStreamException { + writeSimpleElement(writer, elementName, characters, false); + } + + /** + * A helper method to write a link + * + * @param xmlStreamWriter + * the stream to write to + * @param text + * the text of the link + * @param location + * the location of the link + * @throws XMLStreamException + * thrown if there was a problem writing to the stream + */ + protected void writeLink(final XMLStreamWriter xmlStreamWriter, final String text, final String location) + throws XMLStreamException { + xmlStreamWriter.writeStartElement("a"); + xmlStreamWriter.writeAttribute("href", location); + xmlStreamWriter.writeCharacters(text); + xmlStreamWriter.writeEndElement(); + } +} diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/html/HtmlProcessorDocumentationWriter.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/html/HtmlProcessorDocumentationWriter.java new file mode 100644 index 0000000000..26372359df --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/html/HtmlProcessorDocumentationWriter.java @@ -0,0 +1,60 @@ +/* + * 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.html; + +import javax.xml.stream.XMLStreamException; +import javax.xml.stream.XMLStreamWriter; + +import org.apache.nifi.components.ConfigurableComponent; +import org.apache.nifi.processor.Processor; +import org.apache.nifi.processor.Relationship; + +/** + * Writes documentation specific for a Processor. This includes everything for a + * ConfigurableComponent as well as Relationship information. + * + * + */ +public class HtmlProcessorDocumentationWriter extends HtmlDocumentationWriter { + + @Override + protected void writeAdditionalBodyInfo(final ConfigurableComponent configurableComponent, + final XMLStreamWriter xmlStreamWriter) throws XMLStreamException { + final Processor processor = (Processor) configurableComponent; + xmlStreamWriter.writeStartElement("p"); + writeSimpleElement(xmlStreamWriter, "strong", "Relationships: "); + xmlStreamWriter.writeEndElement(); + + if (processor.getRelationships().size() > 0) { + xmlStreamWriter.writeStartElement("table"); + xmlStreamWriter.writeStartElement("tr"); + writeSimpleElement(xmlStreamWriter, "th", "Name"); + writeSimpleElement(xmlStreamWriter, "th", "Description"); + xmlStreamWriter.writeEndElement(); + + for (Relationship relationship : processor.getRelationships()) { + xmlStreamWriter.writeStartElement("tr"); + writeSimpleElement(xmlStreamWriter, "td", relationship.getName()); + writeSimpleElement(xmlStreamWriter, "td", relationship.getDescription()); + xmlStreamWriter.writeEndElement(); + } + xmlStreamWriter.writeEndElement(); + } else { + xmlStreamWriter.writeCharacters("This processor has no relationships."); + } + } +} diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/init/ControllerServiceInitializer.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/init/ControllerServiceInitializer.java new file mode 100644 index 0000000000..123a39c81d --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/init/ControllerServiceInitializer.java @@ -0,0 +1,38 @@ +/* + * 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.ConfigurableComponent; +import org.apache.nifi.controller.ControllerService; +import org.apache.nifi.documentation.ConfigurableComponentInitializer; +import org.apache.nifi.documentation.mock.MockControllerServiceInitializationContext; +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; + controllerService.initialize(new MockControllerServiceInitializationContext()); + } +} diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/init/ProcessorInitializer.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/init/ProcessorInitializer.java new file mode 100644 index 0000000000..a33f7b9acc --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/init/ProcessorInitializer.java @@ -0,0 +1,37 @@ +/* + * 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.ConfigurableComponent; +import org.apache.nifi.documentation.ConfigurableComponentInitializer; +import org.apache.nifi.documentation.mock.MockProcessorInitializationContext; +import org.apache.nifi.processor.Processor; + +/** + * Initializes a Procesor using a MockProcessorInitializationContext + * + * + */ +public class ProcessorInitializer implements ConfigurableComponentInitializer { + + @Override + public void initialize(ConfigurableComponent component) { + Processor processor = (Processor) component; + processor.initialize(new MockProcessorInitializationContext()); + } + +} diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/init/ReportingTaskingInitializer.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/init/ReportingTaskingInitializer.java new file mode 100644 index 0000000000..ff915cfa0b --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/init/ReportingTaskingInitializer.java @@ -0,0 +1,37 @@ +/* + * 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.ConfigurableComponent; +import org.apache.nifi.documentation.ConfigurableComponentInitializer; +import org.apache.nifi.documentation.mock.MockReportingInitializationContext; +import org.apache.nifi.reporting.InitializationException; +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; + reportingTask.initialize(new MockReportingInitializationContext()); + } +} diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/mock/MockControllerServiceInitializationContext.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/mock/MockControllerServiceInitializationContext.java new file mode 100644 index 0000000000..6153a8effe --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/mock/MockControllerServiceInitializationContext.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.documentation.mock; + +import org.apache.nifi.controller.ControllerServiceInitializationContext; +import org.apache.nifi.controller.ControllerServiceLookup; + +/** + * 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 ""; + } + + @Override + public ControllerServiceLookup getControllerServiceLookup() { + return new MockControllerServiceLookup(); + } + +} diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/mock/MockControllerServiceLookup.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/mock/MockControllerServiceLookup.java new file mode 100644 index 0000000000..88d091155e --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/mock/MockControllerServiceLookup.java @@ -0,0 +1,55 @@ +/* + * 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.mock; + +import java.util.Collections; +import java.util.Set; + +import org.apache.nifi.controller.ControllerService; +import org.apache.nifi.controller.ControllerServiceLookup; + +/** + * 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(String serviceIdentifier) { + return null; + } + + @Override + public boolean isControllerServiceEnabled(String serviceIdentifier) { + return false; + } + + @Override + public boolean isControllerServiceEnabled(ControllerService service) { + return false; + } + + @Override + public Set getControllerServiceIdentifiers(Class serviceType) + throws IllegalArgumentException { + return Collections.emptySet(); + } + +} diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/mock/MockProcessorInitializationContext.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/mock/MockProcessorInitializationContext.java new file mode 100644 index 0000000000..48ffecb1ce --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/mock/MockProcessorInitializationContext.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.documentation.mock; + +import org.apache.nifi.controller.ControllerServiceLookup; +import org.apache.nifi.logging.ProcessorLog; +import org.apache.nifi.processor.ProcessorInitializationContext; + +/** + * 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 ""; + } + + @Override + public ProcessorLog getLogger() { + return null; + } + + @Override + public ControllerServiceLookup getControllerServiceLookup() { + return new MockControllerServiceLookup(); + } +} diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/mock/MockReportingInitializationContext.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/mock/MockReportingInitializationContext.java new file mode 100644 index 0000000000..9782077bc9 --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/mock/MockReportingInitializationContext.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.documentation.mock; + +import java.util.concurrent.TimeUnit; + +import org.apache.nifi.controller.ControllerServiceLookup; +import org.apache.nifi.reporting.ReportingInitializationContext; +import org.apache.nifi.scheduling.SchedulingStrategy; + +/** + * A Mock ReportingInitializationContext that can be used to initialize a + * ReportingTask for the purposes of documentation generation. + * + * @author Alligator + * + */ +public class MockReportingInitializationContext implements ReportingInitializationContext { + + @Override + public String getIdentifier() { + return ""; + } + + @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; + } +} diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/resources/apache.license b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/resources/apache.license new file mode 100644 index 0000000000..d7b32e8090 --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/resources/apache.license @@ -0,0 +1,12 @@ +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. \ No newline at end of file diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/java/org/apache/nifi/documentation/example/FullyDocumentedControllerService.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/java/org/apache/nifi/documentation/example/FullyDocumentedControllerService.java new file mode 100644 index 0000000000..c27f3baa51 --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/java/org/apache/nifi/documentation/example/FullyDocumentedControllerService.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.documentation.example; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; + +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.documentation.Tags; +import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.controller.AbstractControllerService; +import org.apache.nifi.processor.util.StandardValidators; + +@CapabilityDescription("A documented controller service that can help you do things") +@Tags({ "one", "two", "three" }) +public class FullyDocumentedControllerService extends AbstractControllerService { + public static final PropertyDescriptor KEYSTORE = new PropertyDescriptor.Builder().name("Keystore Filename") + .description("The fully-qualified filename of the Keystore").defaultValue(null) + .addValidator(StandardValidators.FILE_EXISTS_VALIDATOR).sensitive(false).build(); + public static final PropertyDescriptor KEYSTORE_TYPE = new PropertyDescriptor.Builder().name("Keystore Type") + .description("The Type of the Keystore").allowableValues("JKS", "PKCS12") + .addValidator(StandardValidators.NON_EMPTY_VALIDATOR).defaultValue("JKS").sensitive(false).build(); + public static final PropertyDescriptor KEYSTORE_PASSWORD = new PropertyDescriptor.Builder() + .name("Keystore Password").defaultValue(null).description("The password for the Keystore") + .addValidator(StandardValidators.NON_EMPTY_VALIDATOR).sensitive(true).build(); + + private static final List properties; + + static { + List props = new ArrayList<>(); + props.add(KEYSTORE); + props.add(KEYSTORE_PASSWORD); + props.add(KEYSTORE_TYPE); + properties = Collections.unmodifiableList(props); + } + + @Override + protected List getSupportedPropertyDescriptors() { + return properties; + } + +} diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/java/org/apache/nifi/documentation/example/FullyDocumentedProcessor.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/java/org/apache/nifi/documentation/example/FullyDocumentedProcessor.java new file mode 100644 index 0000000000..2a001779ad --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/java/org/apache/nifi/documentation/example/FullyDocumentedProcessor.java @@ -0,0 +1,100 @@ +/* + * 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.example; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.Set; + +import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.processor.AbstractProcessor; +import org.apache.nifi.processor.ProcessContext; +import org.apache.nifi.processor.ProcessSession; +import org.apache.nifi.processor.ProcessorInitializationContext; +import org.apache.nifi.processor.Relationship; +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.documentation.Tags; +import org.apache.nifi.processor.exception.ProcessException; +import org.apache.nifi.processor.util.StandardValidators; + +@Tags({ "one", "two", "three" }) +@CapabilityDescription("This is a processor that is used to test documentation.") +public class FullyDocumentedProcessor extends AbstractProcessor { + + public static final PropertyDescriptor DIRECTORY = new PropertyDescriptor.Builder().name("Input Directory") + .description("The input directory from which to pull files").required(true) + .addValidator(StandardValidators.createDirectoryExistsValidator(true, false)) + .expressionLanguageSupported(true).build(); + + public static final PropertyDescriptor RECURSE = new PropertyDescriptor.Builder().name("Recurse Subdirectories") + .description("Indicates whether or not to pull files from subdirectories").required(true) + .allowableValues("true", "false").defaultValue("true").build(); + + public static final PropertyDescriptor POLLING_INTERVAL = new PropertyDescriptor.Builder().name("Polling Interval") + .description("Indicates how long to wait before performing a directory listing").required(true) + .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR).defaultValue("0 sec").build(); + + public static final PropertyDescriptor OPTIONAL_PROPERTY = new PropertyDescriptor.Builder() + .name("Optional Property").description("This is a property you can use or not").required(false).build(); + + public static final Relationship REL_SUCCESS = new Relationship.Builder().name("success") + .description("Successful files").build(); + public static final Relationship REL_FAILURE = new Relationship.Builder().name("failure") + .description("Failing files").build(); + + private List properties; + private Set relationships; + + @Override + protected void init(ProcessorInitializationContext context) { + final List properties = new ArrayList<>(); + properties.add(DIRECTORY); + properties.add(RECURSE); + properties.add(POLLING_INTERVAL); + properties.add(OPTIONAL_PROPERTY); + this.properties = Collections.unmodifiableList(properties); + + final Set relationships = new HashSet<>(); + relationships.add(REL_SUCCESS); + relationships.add(REL_FAILURE); + this.relationships = Collections.unmodifiableSet(relationships); + } + + @Override + protected List getSupportedPropertyDescriptors() { + return properties; + } + + @Override + public Set getRelationships() { + return relationships; + } + + @Override + public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException { + + } + + @Override + protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(String propertyDescriptorName) { + return new PropertyDescriptor.Builder().name(propertyDescriptorName) + .description("This is a property you can use or not").dynamic(true).build(); + } + +} diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/java/org/apache/nifi/documentation/example/FullyDocumentedReportingTask.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/java/org/apache/nifi/documentation/example/FullyDocumentedReportingTask.java new file mode 100644 index 0000000000..275905b1a4 --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/java/org/apache/nifi/documentation/example/FullyDocumentedReportingTask.java @@ -0,0 +1,50 @@ +/* + * 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.example; + +import java.util.ArrayList; +import java.util.List; + +import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.documentation.Tags; +import org.apache.nifi.reporting.AbstractReportingTask; +import org.apache.nifi.reporting.ReportingContext; + +@CapabilityDescription("A helper reporting task to do...") +@Tags({ "first", "second", "third" }) +public class FullyDocumentedReportingTask extends AbstractReportingTask { + + public static final PropertyDescriptor SHOW_DELTAS = new PropertyDescriptor.Builder() + .name("Show Deltas") + .description( + "Specifies whether or not to show the difference in values between the current status and the previous status") + .required(true).allowableValues("true", "false").defaultValue("true").build(); + + @Override + public final List getSupportedPropertyDescriptors() { + final List descriptors = new ArrayList<>(); + descriptors.add(SHOW_DELTAS); + return descriptors; + } + + @Override + public void onTrigger(ReportingContext context) { + // TODO Auto-generated method stub + + } +} diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/java/org/apache/nifi/documentation/html/HtmlDocumentationWriterTest.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/java/org/apache/nifi/documentation/html/HtmlDocumentationWriterTest.java new file mode 100644 index 0000000000..f685f39b74 --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/java/org/apache/nifi/documentation/html/HtmlDocumentationWriterTest.java @@ -0,0 +1,56 @@ +/* + * 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.html; + +import java.io.IOException; + +import org.apache.nifi.controller.ControllerService; +import org.apache.nifi.documentation.DocumentationWriter; +import org.apache.nifi.documentation.example.FullyDocumentedControllerService; +import org.apache.nifi.documentation.example.FullyDocumentedReportingTask; +import org.apache.nifi.documentation.mock.MockControllerServiceInitializationContext; +import org.apache.nifi.documentation.mock.MockReportingInitializationContext; +import org.apache.nifi.reporting.InitializationException; +import org.apache.nifi.reporting.ReportingTask; +import org.junit.Test; + +public class HtmlDocumentationWriterTest { + + @Test + public void testDocumentControllerService() throws InitializationException, IOException { + + ControllerService controllerService = new FullyDocumentedControllerService(); + controllerService.initialize(new MockControllerServiceInitializationContext()); + + DocumentationWriter writer = new HtmlDocumentationWriter(); + + writer.write(controllerService, System.out, false); + + } + + @Test + public void testDocumentReportingTask() throws InitializationException, IOException { + + ReportingTask reportingTask = new FullyDocumentedReportingTask(); + reportingTask.initialize(new MockReportingInitializationContext()); + + DocumentationWriter writer = new HtmlDocumentationWriter(); + + writer.write(reportingTask, System.out, false); + } + +} diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/java/org/apache/nifi/documentation/html/ProcessorDocumentationWriterTest.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/java/org/apache/nifi/documentation/html/ProcessorDocumentationWriterTest.java new file mode 100644 index 0000000000..faf66b58a9 --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/java/org/apache/nifi/documentation/html/ProcessorDocumentationWriterTest.java @@ -0,0 +1,69 @@ +/* + * 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.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.documentation.CapabilityDescription; +import org.apache.nifi.documentation.DocumentationWriter; +import org.apache.nifi.documentation.example.FullyDocumentedProcessor; +import org.apache.nifi.documentation.mock.MockProcessorInitializationContext; +import org.junit.Test; + +public class ProcessorDocumentationWriterTest { + + @Test + public void testFullyDocumentedProcessor() throws IOException { + FullyDocumentedProcessor processor = new FullyDocumentedProcessor(); + processor.initialize(new MockProcessorInitializationContext()); + + DocumentationWriter writer = new HtmlProcessorDocumentationWriter(); + + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + + writer.write(processor, baos, false); + + String results = new String(baos.toByteArray()); + + assertContains(results, FullyDocumentedProcessor.DIRECTORY.getDisplayName()); + assertContains(results, FullyDocumentedProcessor.DIRECTORY.getDescription()); + assertContains(results, FullyDocumentedProcessor.OPTIONAL_PROPERTY.getDisplayName()); + assertContains(results, FullyDocumentedProcessor.OPTIONAL_PROPERTY.getDescription()); + assertContains(results, FullyDocumentedProcessor.POLLING_INTERVAL.getDisplayName()); + assertContains(results, FullyDocumentedProcessor.POLLING_INTERVAL.getDescription()); + assertContains(results, FullyDocumentedProcessor.POLLING_INTERVAL.getDefaultValue()); + assertContains(results, FullyDocumentedProcessor.RECURSE.getDisplayName()); + assertContains(results, FullyDocumentedProcessor.RECURSE.getDescription()); + + assertContains(results, FullyDocumentedProcessor.REL_SUCCESS.getName()); + assertContains(results, FullyDocumentedProcessor.REL_SUCCESS.getDescription()); + assertContains(results, FullyDocumentedProcessor.REL_FAILURE.getName()); + assertContains(results, FullyDocumentedProcessor.REL_FAILURE.getDescription()); + + assertNotContains(results, "iconSecure.png"); + assertContains(results, FullyDocumentedProcessor.class.getAnnotation(CapabilityDescription.class).value()); + assertNotContains(results, "This component has no required or optional properties."); + assertNotContains(results, "No description provided."); + assertNotContains(results, "No Tags provided."); + assertNotContains(results, "Additional Details..."); + } + +} diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/java/org/apache/nifi/documentation/html/XmlValidator.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/java/org/apache/nifi/documentation/html/XmlValidator.java new file mode 100644 index 0000000000..6cb7499628 --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/java/org/apache/nifi/documentation/html/XmlValidator.java @@ -0,0 +1,50 @@ +/* + * 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.html; + +import java.io.IOException; +import java.io.StringReader; + +import javax.xml.parsers.DocumentBuilderFactory; +import javax.xml.parsers.ParserConfigurationException; + +import org.junit.Assert; +import org.xml.sax.InputSource; +import org.xml.sax.SAXException; + +/** + * A helper class to validate xml documents. + * + * + */ +public class XmlValidator { + public static void assertXmlValid(String xml) { + try { + DocumentBuilderFactory.newInstance().newDocumentBuilder().parse(new InputSource(new StringReader(xml))); + } catch (SAXException | IOException | ParserConfigurationException e) { + Assert.fail(e.getMessage()); + } + } + + public static void assertContains(String original, String subword) { + Assert.assertTrue(original + " did not contain: " + subword, original.contains(subword)); + } + + public static void assertNotContains(String original, String subword) { + Assert.assertFalse(original + " did contain: " + subword, original.contains(subword)); + } +} diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-runtime/pom.xml b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-runtime/pom.xml index b75c35e6c0..78f80df684 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-runtime/pom.xml +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-runtime/pom.xml @@ -31,6 +31,11 @@ org.apache.nifi nifi-properties compile + + + org.apache.nifi + nifi-documentation + compile org.slf4j diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-runtime/src/main/java/org/apache/nifi/NiFi.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-runtime/src/main/java/org/apache/nifi/NiFi.java index ed7c3298dd..e166f8e6de 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-runtime/src/main/java/org/apache/nifi/NiFi.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-runtime/src/main/java/org/apache/nifi/NiFi.java @@ -31,6 +31,7 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; +import org.apache.nifi.documentation.DocGenerator; import org.apache.nifi.nar.ExtensionManager; import org.apache.nifi.nar.ExtensionMapping; import org.apache.nifi.nar.NarClassLoaders; @@ -117,6 +118,8 @@ public class NiFi { // discover the extensions ExtensionManager.discoverExtensions(); ExtensionManager.logClassLoaderMapping(); + + DocGenerator.generate(properties); // load the server from the framework classloader Thread.currentThread().setContextClassLoader(frameworkClassLoader); diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/pom.xml b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/pom.xml index c38612856a..b5a1d542c8 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/pom.xml +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/pom.xml @@ -40,6 +40,7 @@ nifi-administration nifi-web nifi-resources + nifi-documentation diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/pom.xml b/nifi/nifi-nar-bundles/nifi-framework-bundle/pom.xml index 413bfca3d2..fcb1243f05 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/pom.xml +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/pom.xml @@ -109,6 +109,11 @@ nifi-web-security 0.0.2-incubating-SNAPSHOT + + org.apache.nifi + nifi-documentation + 0.0.2-incubating-SNAPSHOT + From 220fcac169ce180ef0ad34a3d0080a1b03e337a7 Mon Sep 17 00:00:00 2001 From: danbress Date: Sat, 31 Jan 2015 12:29:07 -0500 Subject: [PATCH 02/26] NIFI-280 rewording column headers and warning message --- .../main/java/org/apache/nifi/documentation/DocGenerator.java | 2 +- .../nifi/documentation/html/HtmlDocumentationWriter.java | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/DocGenerator.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/DocGenerator.java index 9a373565db..157d95edf5 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/DocGenerator.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/DocGenerator.java @@ -111,7 +111,7 @@ public class DocGenerator { final File baseDocumenationFile = new File(directory, "index.html"); if (baseDocumenationFile.exists()) { - logger.warn(baseDocumenationFile + " already exists! Overwriting!"); + logger.warn(baseDocumenationFile + " already exists, overwriting!"); } try (final OutputStream output = new BufferedOutputStream(new FileOutputStream(baseDocumenationFile))) { diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/html/HtmlDocumentationWriter.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/html/HtmlDocumentationWriter.java index bf9bd131b0..f2ce158a43 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/html/HtmlDocumentationWriter.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/html/HtmlDocumentationWriter.java @@ -274,8 +274,8 @@ public class HtmlDocumentationWriter implements DocumentationWriter { xmlStreamWriter.writeStartElement("tr"); writeSimpleElement(xmlStreamWriter, "th", "Name"); writeSimpleElement(xmlStreamWriter, "th", "Description"); - writeSimpleElement(xmlStreamWriter, "th", "Default Value"); - writeSimpleElement(xmlStreamWriter, "th", "Valid Values"); + writeSimpleElement(xmlStreamWriter, "th", "Default"); + writeSimpleElement(xmlStreamWriter, "th", "Values"); xmlStreamWriter.writeStartElement("th"); writeLink(xmlStreamWriter, "EL", "../../html/expression-language-guide.html"); xmlStreamWriter.writeEndElement(); From 0fa553ac0265f96cb5c2a5f42bc760aa0c65b590 Mon Sep 17 00:00:00 2001 From: danbress Date: Sat, 31 Jan 2015 12:56:25 -0500 Subject: [PATCH 03/26] NIFI-280 - Modifing existing existing documentation removing index.html and renaming to additionalDetails.html if there is relevent information --- .../{index.html => additionalDetails.html} | 35 -- .../additionalDetails.html | 57 +++ .../index.html | 162 -------- .../additionalDetails.html | 32 ++ .../index.html | 150 -------- .../additionalDetails.html | 51 +++ .../index.html | 159 -------- .../additionalDetails.html | 78 ++++ .../index.html | 173 --------- .../additionalDetails.html} | 43 +-- .../index.html | 189 ---------- .../index.html | 63 ---- .../additionalDetails.html | 68 ++++ .../index.html | 166 --------- .../index.html | 116 ------ .../index.html | 65 ---- .../additionalDetails.html | 58 +++ .../index.html | 147 -------- .../{index.html => additionalDetails.html} | 46 --- .../index.html | 97 ----- .../index.html | 160 -------- .../additionalDetails.html | 34 ++ .../index.html | 135 ------- .../additionalDetails.html | 161 ++++++++ .../index.html | 311 ---------------- .../additionalDetails.html | 57 +++ .../index.html | 111 ------ .../index.html | 64 ---- .../additionalDetails.html | 74 ++++ .../index.html | 227 ------------ .../additionalDetails.html | 79 ++++ .../index.html | 186 ---------- .../additionalDetails.html | 45 +++ .../index.html | 143 -------- .../index.html | 118 ------ .../index.html | 122 ------ .../additionalDetails.html | 80 ++++ .../index.html | 250 ------------- .../additionalDetails.html | 35 ++ .../index.html | 88 ----- .../additionalDetails.html | 35 ++ .../index.html | 89 ----- .../{index.html => additionalDetails.html} | 42 +-- .../additionalDetails.html | 64 ++++ .../index.html | 181 --------- .../index.html | 86 ----- .../index.html | 144 -------- .../index.html | 80 ---- .../additionalDetails.html | 115 ++++++ .../index.html | 347 ------------------ .../index.html | 64 ---- .../additionalDetails.html | 49 +++ .../index.html | 143 -------- .../additionalDetails.html | 48 +++ .../index.html | 187 ---------- .../index.html | 114 ------ .../additionalDetails.html | 66 ++++ .../index.html | 283 -------------- .../index.html | 109 ------ .../index.html | 152 -------- .../additionalDetails.html | 34 ++ .../index.html | 281 -------------- .../index.html | 91 ----- .../index.html | 114 ------ .../additionalDetails.html | 46 +++ .../index.html | 110 ------ .../index.html | 82 ----- .../index.html | 85 ----- .../additionalDetails.html | 45 +++ .../index.html | 100 ----- .../{index.html => additionalDetails.html} | 43 +-- .../{index.html => additionalDetails.html} | 42 +-- .../{index.html => additionalDetails.html} | 51 +-- .../index.html | 64 ---- .../index.html | 63 ---- .../{index.html => additionalDetails.html} | 61 +-- .../{index.html => additionalDetails.html} | 15 - .../index.html | 58 --- .../index.html | 77 ---- .../{index.html => additionalDetails.html} | 24 -- .../{index.html => additionalDetails.html} | 46 --- .../{index.html => additionalDetails.html} | 20 - .../{index.html => additionalDetails.html} | 50 --- .../{index.html => additionalDetails.html} | 14 - .../{index.html => additionalDetails.html} | 0 85 files changed, 1434 insertions(+), 7015 deletions(-) rename nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/resources/docs/org.apache.nifi.processors.hadoop.CreateHadoopSequenceFile/{index.html => additionalDetails.html} (68%) create mode 100644 nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/resources/docs/org.apache.nifi.processors.hadoop.GetHDFS/additionalDetails.html delete mode 100644 nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/resources/docs/org.apache.nifi.processors.hadoop.GetHDFS/index.html create mode 100644 nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/resources/docs/org.apache.nifi.processors.hadoop.GetHDFSSequenceFile/additionalDetails.html delete mode 100644 nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/resources/docs/org.apache.nifi.processors.hadoop.GetHDFSSequenceFile/index.html create mode 100644 nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/resources/docs/org.apache.nifi.processors.hadoop.PutHDFS/additionalDetails.html delete mode 100644 nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/resources/docs/org.apache.nifi.processors.hadoop.PutHDFS/index.html create mode 100644 nifi/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/resources/docs/org.apache.nifi.processors.kafka.GetKafka/additionalDetails.html delete mode 100644 nifi/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/resources/docs/org.apache.nifi.processors.kafka.GetKafka/index.html rename nifi/nifi-nar-bundles/{nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.ValidateXml/index.html => nifi-kafka-bundle/nifi-kafka-processors/src/main/resources/docs/org.apache.nifi.processors.kafka.PutKafka/additionalDetails.html} (50%) delete mode 100644 nifi/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/resources/docs/org.apache.nifi.processors.kafka.PutKafka/index.html delete mode 100644 nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.Base64EncodeContent/index.html create mode 100644 nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.CompressContent/additionalDetails.html delete mode 100644 nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.CompressContent/index.html delete mode 100644 nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.ControlRate/index.html delete mode 100644 nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.ConvertCharacterSet/index.html create mode 100644 nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.DetectDuplicate/additionalDetails.html delete mode 100644 nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.DetectDuplicate/index.html rename nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.DistributeLoad/{index.html => additionalDetails.html} (54%) delete mode 100644 nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.EncryptContent/index.html delete mode 100644 nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.EvaluateRegularExpression/index.html create mode 100644 nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.EvaluateXPath/additionalDetails.html delete mode 100644 nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.EvaluateXPath/index.html create mode 100644 nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.EvaluateXQuery/additionalDetails.html delete mode 100644 nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.EvaluateXQuery/index.html create mode 100644 nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.ExecuteStreamCommand/additionalDetails.html delete mode 100644 nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.ExecuteStreamCommand/index.html delete mode 100644 nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.GenerateFlowFile/index.html create mode 100644 nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.GetFTP/additionalDetails.html delete mode 100644 nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.GetFTP/index.html create mode 100644 nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.GetFile/additionalDetails.html delete mode 100644 nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.GetFile/index.html create mode 100644 nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.GetHTTP/additionalDetails.html delete mode 100644 nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.GetHTTP/index.html delete mode 100644 nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.GetJMSQueue/index.html delete mode 100644 nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.GetJMSTopic/index.html create mode 100644 nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.GetSFTP/additionalDetails.html delete mode 100644 nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.GetSFTP/index.html create mode 100644 nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.HashAttribute/additionalDetails.html delete mode 100644 nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.HashAttribute/index.html create mode 100644 nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.HashContent/additionalDetails.html delete mode 100644 nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.HashContent/index.html rename nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.IdentifyMimeType/{index.html => additionalDetails.html} (73%) create mode 100644 nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.InvokeHTTP/additionalDetails.html delete mode 100644 nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.InvokeHTTP/index.html delete mode 100644 nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.ListenHTTP/index.html delete mode 100644 nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.ListenUDP/index.html delete mode 100644 nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.LogAttribute/index.html create mode 100644 nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.MergeContent/additionalDetails.html delete mode 100644 nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.MergeContent/index.html delete mode 100644 nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.ModifyBytes/index.html create mode 100644 nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.MonitorActivity/additionalDetails.html delete mode 100644 nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.MonitorActivity/index.html create mode 100644 nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.PostHTTP/additionalDetails.html delete mode 100644 nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.PostHTTP/index.html delete mode 100644 nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.PutEmail/index.html create mode 100644 nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.PutFTP/additionalDetails.html delete mode 100644 nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.PutFTP/index.html delete mode 100644 nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.PutFile/index.html delete mode 100644 nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.PutJMS/index.html create mode 100644 nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.PutSFTP/additionalDetails.html delete mode 100644 nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.PutSFTP/index.html delete mode 100644 nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.ReplaceText/index.html delete mode 100644 nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.ReplaceTextWithMapping/index.html create mode 100644 nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.RouteOnAttribute/additionalDetails.html delete mode 100644 nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.RouteOnAttribute/index.html delete mode 100644 nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.RouteOnContent/index.html delete mode 100644 nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.ScanAttribute/index.html create mode 100644 nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.ScanContent/additionalDetails.html delete mode 100644 nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.ScanContent/index.html rename nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.SegmentContent/{index.html => additionalDetails.html} (68%) rename nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.SplitContent/{index.html => additionalDetails.html} (60%) rename nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.SplitText/{index.html => additionalDetails.html} (54%) delete mode 100644 nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.SplitXml/index.html delete mode 100644 nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.TransformXml/index.html rename nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.UnpackContent/{index.html => additionalDetails.html} (66%) rename nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-reporting-tasks/src/main/resources/docs/org.apache.nifi.controller.ControllerStatusReportingTask/{index.html => additionalDetails.html} (87%) delete mode 100644 nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-reporting-tasks/src/main/resources/docs/org.apache.nifi.controller.MonitorDiskUsage/index.html delete mode 100644 nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-reporting-tasks/src/main/resources/docs/org.apache.nifi.controller.MonitorMemory/index.html rename nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-reporting-tasks/src/main/resources/docs/org.apache.nifi.reporting.ganglia.StandardGangliaReporter/{index.html => additionalDetails.html} (76%) rename nifi/nifi-nar-bundles/nifi-standard-services/nifi-distributed-cache-services-bundle/nifi-distributed-cache-client-service/src/main/resources/docs/org.apache.nifi.distributed.cache.client.DistributedMapCacheClientService/{index.html => additionalDetails.html} (57%) rename nifi/nifi-nar-bundles/nifi-standard-services/nifi-distributed-cache-services-bundle/nifi-distributed-cache-client-service/src/main/resources/docs/org.apache.nifi.distributed.cache.client.DistributedSetCacheClientService/{index.html => additionalDetails.html} (70%) mode change 100755 => 100644 rename nifi/nifi-nar-bundles/nifi-standard-services/nifi-distributed-cache-services-bundle/nifi-distributed-cache-server/src/main/resources/docs/org.apache.nifi.distributed.cache.server.map.DistributedMapCacheServer/{index.html => additionalDetails.html} (54%) rename nifi/nifi-nar-bundles/nifi-standard-services/nifi-ssl-context-bundle/nifi-ssl-context-service/src/main/resources/docs/org.apache.nifi.ssl.StandardSSLContextService/{index.html => additionalDetails.html} (77%) rename nifi/nifi-nar-bundles/nifi-update-attribute-bundle/nifi-update-attribute-processor/src/main/resources/docs/org.apache.nifi.processors.attributes.UpdateAttribute/{index.html => additionalDetails.html} (100%) diff --git a/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/resources/docs/org.apache.nifi.processors.hadoop.CreateHadoopSequenceFile/index.html b/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/resources/docs/org.apache.nifi.processors.hadoop.CreateHadoopSequenceFile/additionalDetails.html similarity index 68% rename from nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/resources/docs/org.apache.nifi.processors.hadoop.CreateHadoopSequenceFile/index.html rename to nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/resources/docs/org.apache.nifi.processors.hadoop.CreateHadoopSequenceFile/additionalDetails.html index ff1709ebdd..d9403822b8 100644 --- a/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/resources/docs/org.apache.nifi.processors.hadoop.CreateHadoopSequenceFile/index.html +++ b/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/resources/docs/org.apache.nifi.processors.hadoop.CreateHadoopSequenceFile/additionalDetails.html @@ -43,41 +43,6 @@ NOTE: The value portion of a key/value pair is loaded into memory. While there is a max size limit of 2GB, this could cause memory issues if there are too many concurrent tasks and the flow file sizes are large. -

Properties:

-

In the list below, the names of required properties appear in bold. Any other properties (not in bold) are - considered optional. If a property has a default value, it is indicated. If a property supports the use of the - NiFi Expression Language (or simply, "expression language"), that is also indicated.

-
    -
  • Hadoop Configuration Resources -
      -
    • A file or comma separated list of files which contains the Hadoop file system configuration. - Without this, Hadoop will search the classpath for a 'core-site.xml' and 'hdfs-site.xml' file or will - revert to a default configuration.
    • -
    • Default value: none
    • -
    -
  • -
  • compression type -
      -
    • Type of compression to use when creating Sequence File.
    • -
    • Default value: none
    • -
    -
  • -
- -

Relationships:

-
    -
  • success -
      -
    • Generated Sequence Files are sent to this relationship.
    • -
    -
  • -
  • fail -
      -
    • Incoming files that failed to generate a Sequence File are sent to this relationship.
    • -
    -
  • -
-

See Also:

diff --git a/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/resources/docs/org.apache.nifi.processors.hadoop.GetHDFS/additionalDetails.html b/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/resources/docs/org.apache.nifi.processors.hadoop.GetHDFS/additionalDetails.html new file mode 100644 index 0000000000..0060c2c1f6 --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/resources/docs/org.apache.nifi.processors.hadoop.GetHDFS/additionalDetails.html @@ -0,0 +1,57 @@ + + + + + + GetHDFS + + + + + +

+ Modifies Attributes: +

+ + + + + + + + + + + + + + + + + +
Attribute NameDescription
filenameThe name of the file that was read from HDFS.
pathThe path is set to the relative path of the file's directory on HDFS. For example, if the Directory + property is set to /tmp, then files picked up from /tmp will have the path attribute set to + "./". If the Recurse Subdirectories property is set to true + and a file is picked up from /tmp/abc/1/2/3, then the path attribute will be set to "abc/1/2/3". +
+

+ See Also: +

+ + + diff --git a/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/resources/docs/org.apache.nifi.processors.hadoop.GetHDFS/index.html b/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/resources/docs/org.apache.nifi.processors.hadoop.GetHDFS/index.html deleted file mode 100644 index 162e7a1417..0000000000 --- a/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/resources/docs/org.apache.nifi.processors.hadoop.GetHDFS/index.html +++ /dev/null @@ -1,162 +0,0 @@ - - - - - - GetHDFS - - - - - - -

Description:

-

This processor reads files from an HDFS cluster into NiFi FlowFiles.

- -

- Modifies Attributes: -

- - - - - - - - - - - - - - - - - -
Attribute NameDescription
filenameThe name of the file that was read from HDFS.
pathThe path is set to the relative path of the file's directory on HDFS. For example, if the Directory - property is set to /tmp, then files picked up from /tmp will have the path attribute set to - "./". If the Recurse Subdirectories property is set to true - and a file is picked up from /tmp/abc/1/2/3, then the path attribute will be set to "abc/1/2/3". -
- -

Properties:

-

In the list below, the names of required properties appear in bold. Any other properties (not in bold) are - considered optional. If a property has a default value, it is indicated. If a property supports the use of the - NiFi Expression Language (or simply, "expression language"), that is also indicated.

-
    -
  • Hadoop Configuration Resources -
      -
    • A file or comma separated list of files which contains the Hadoop file system configuration. - Without this, Hadoop will search the classpath for a 'core-site.xml' and 'hdfs-site.xml' file or will - revert to a default configuration.
    • -
    • Default value: none
    • -
    -
  • -
  • Directory -
      -
    • The HDFS directory from which FlowFile content should be read.
    • -
    • Default value: none
    • -
  • -
  • Recurse Subdirectories -
      -
    • A Boolean value (true/false), when true will pull files from subdirectories of the HDFS Directory. -
    • -
    • Default value: true
    • -
  • -
  • Keep Source File -
      -
    • A Boolean value (true/false), indicates whether to keep (true) or delete (false) the file from HDFS - after it has been successfully transferred.
    • -
    • Default value: false
    • -
  • -
  • File Filter Regex -
      -
    • A Java Regular Expression for filtering Filenames; if a filter is supplied then only files whose - names match that Regular Expression will be fetched, otherwise all files will be fetched.
    • -
    • Default value: none
    • -
    -
  • -
  • Filter Match Name Only -
      -
    • A Boolean value (true/false), when true File Filter Regex will match on just the filename, - otherwise subdirectory names will be included with filename in the regex comparison. -
    • -
    • Default value: true
    • -
  • -
  • Ignore Dotted Files -
      -
    • A Boolean value (true/false), when true files whose names begin with a dot (".") will not - be fetched.
    • -
    • Default value: true
    • -
  • -
  • Minimum File Age -
      -
    • The minimum age that a file must be in order to be fetched; any file that is younger than this - amount of time (based on last modification time) will be ignored. The value must be a non-negative - integer and be followed by a time unit, such as nanos, millis, secs, mins, hrs, days.
    • -
    • Default value: 0 sec
    • -
  • -
  • Maximum File Age -
      -
    • The maximum age that a file must be in order to be fetched; any file that is older than this amount - of time (based on last modification time) will be ignored. The value must be a non-negative integer, - followed by a time unit, such as nanos, millis, secs, mins, hrs, days. Cannot be less than 100 millis.
    • -
    • Default value: none
    • -
    -
  • -
  • Polling Interval -
      -
    • Indicates how long to wait between performing directory listings. The value must be a non-negative - integer and be followed by a time unit, such as nanos, millis, secs, mins, hrs, days.
    • -
    • Default value: 0 sec
    • -
  • -
  • Batch Size -
      -
    • The maximum number of files to pull in each iteration, based on configured run schedule.
    • -
    • Default value: 100
    • -
  • -
  • IO Buffer Size -
      -
    • Amount of memory to use to buffer file contents during IO. This is a data size integer that must - include units of B, KB, MB, GB, or TB. This overrides the Hadoop Configuration.
    • -
    • Default value: none
    • -
    -
  • -
- -

Relationships:

-
    -
  • success -
      -
    • All files retrieved from HDFS are transferred to this relationship.
    • -
    -
  • -
  • passthrough -
      -
    • If this processor has an input queue for some reason, then FlowFiles arriving on that input are - transferred to this relationship.
    • -
    -
  • -
- -

- See Also: -

- - - diff --git a/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/resources/docs/org.apache.nifi.processors.hadoop.GetHDFSSequenceFile/additionalDetails.html b/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/resources/docs/org.apache.nifi.processors.hadoop.GetHDFSSequenceFile/additionalDetails.html new file mode 100644 index 0000000000..ca0b7cdfbe --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/resources/docs/org.apache.nifi.processors.hadoop.GetHDFSSequenceFile/additionalDetails.html @@ -0,0 +1,32 @@ + + + + + + GetHDFSSequenceFile + + + + + +

+ See Also: +

+ + + diff --git a/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/resources/docs/org.apache.nifi.processors.hadoop.GetHDFSSequenceFile/index.html b/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/resources/docs/org.apache.nifi.processors.hadoop.GetHDFSSequenceFile/index.html deleted file mode 100644 index 71f5fa7f4a..0000000000 --- a/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/resources/docs/org.apache.nifi.processors.hadoop.GetHDFSSequenceFile/index.html +++ /dev/null @@ -1,150 +0,0 @@ - - - - - - GetHDFSSequenceFile - - - - - - -

Description:

-

This processor is used to pull files from HDFS. The files being pulled in MUST be SequenceFile - formatted files. The processor creates a flow file for each key/value entry in the ingested SequenceFile. - The created flow file's content depends on the value of the optional configuration property FlowFile Content. Currently, - there are two choices: VALUE ONLY and KEY VALUE PAIR. With the prior, only the SequenceFile value element is - written to the flow file contents. With the latter, the SequenceFile key and value are written to the flow file - contents as serialized objects; the format is key length (int), key(String), value length(int), value(bytes). The default is - VALUE ONLY.

- NOTE: This processor loads the entire value entry into memory. While the size limit for a value entry is 2GB, this will cause - memory problems if there are too many concurrent tasks and the data being ingested is large. - -

Properties:

-

In the list below, the names of required properties appear in bold. Any other properties (not in bold) are - considered optional. If a property has a default value, it is indicated. If a property supports the use of the - NiFi Expression Language (or simply, "expression language"), that is also indicated.

-
    -
  • Hadoop Configuration Resources -
      -
    • A file or comma separated list of files which contains the Hadoop file system configuration. - Without this, Hadoop will search the classpath for a 'core-site.xml' and 'hdfs-site.xml' file or will - revert to a default configuration.
    • -
    • Default value: none
    • -
    -
  • -
  • FlowFile Content -
      -
    • Indicate if the content is to be both the key and value of the Sequence File, or just the value.
    • -
    • Default value: VALUE ONLY
    • -
    -
  • -
  • Directory -
      -
    • The HDFS directory from which FlowFile content should be read.
    • -
    • Default value: none
    • -
  • -
  • Recurse Subdirectories -
      -
    • A Boolean value (true/false), when true will pull files from subdirectories of the HDFS Directory. -
    • -
    • Default value: true
    • -
  • -
  • Keep Source File -
      -
    • A Boolean value (true/false), indicates whether to keep (true) or delete (false) the file from HDFS - after it has been successfully transferred.
    • -
    • Default value: false
    • -
  • -
  • File Filter Regex -
      -
    • A Java Regular Expression for filtering Filenames; if a filter is supplied then only files whose - names match that Regular Expression will be fetched, otherwise all files will be fetched.
    • -
    • Default value: none
    • -
    -
  • -
  • Filter Match Name Only -
      -
    • A Boolean value (true/false), when true File Filter Regex will match on just the filename, - otherwise subdirectory names will be included with filename in the regex comparison. -
    • -
    • Default value: true
    • -
  • -
  • Ignore Dotted Files -
      -
    • A Boolean value (true/false), when true files whose names begin with a dot (".") will not - be fetched.
    • -
    • Default value: true
    • -
  • -
  • Minimum File Age -
      -
    • The minimum age that a file must be in order to be fetched; any file that is younger than this - amount of time (based on last modification time) will be ignored. The value must be a non-negative - integer and be followed by a time unit, such as nanos, millis, secs, mins, hrs, days.
    • -
    • Default value: 0 sec
    • -
  • -
  • Maximum File Age -
      -
    • The maximum age that a file must be in order to be fetched; any file that is older than this amount - of time (based on last modification time) will be ignored. The value must be a non-negative integer, - followed by a time unit, such as nanos, millis, secs, mins, hrs, days. Cannot be less than 100 millis.
    • -
    • Default value: none
    • -
    -
  • -
  • Polling Interval -
      -
    • Indicates how long to wait between performing directory listings. The value must be a non-negative - integer and be followed by a time unit, such as nanos, millis, secs, mins, hrs, days.
    • -
    • Default value: 0 sec
    • -
  • -
  • Batch Size -
      -
    • The maximum number of files to pull in each iteration, based on configured run schedule.
    • -
    • Default value: 100
    • -
  • -
  • IO Buffer Size -
      -
    • Amount of memory to use to buffer file contents during IO. This is a data size integer that must - include units of B, KB, MB, GB, or TB. This overrides the Hadoop Configuration.
    • -
    • Default value: none
    • -
    -
  • -
- -

Relationships:

-
    -
  • success -
      -
    • All files retrieved from HDFS are transferred to this relationship.
    • -
    -
  • -
  • passthrough -
      -
    • If this processor has an input queue for some reason, then FlowFiles arriving on that input are - transferred to this relationship.
    • -
    -
  • -
- -

- See Also: -

- - - diff --git a/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/resources/docs/org.apache.nifi.processors.hadoop.PutHDFS/additionalDetails.html b/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/resources/docs/org.apache.nifi.processors.hadoop.PutHDFS/additionalDetails.html new file mode 100644 index 0000000000..1a02d1085c --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/resources/docs/org.apache.nifi.processors.hadoop.PutHDFS/additionalDetails.html @@ -0,0 +1,51 @@ + + + + + + PutHDFS + + + + + + +

+ Uses Attributes: +

+ + + + + + + + + + + + + +
Attribute NameDescription
filenameThe name of the file written to HDFS comes from the value of this attribute.
+ +

+ See Also: +

+ + + diff --git a/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/resources/docs/org.apache.nifi.processors.hadoop.PutHDFS/index.html b/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/resources/docs/org.apache.nifi.processors.hadoop.PutHDFS/index.html deleted file mode 100644 index cacdd721cb..0000000000 --- a/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/resources/docs/org.apache.nifi.processors.hadoop.PutHDFS/index.html +++ /dev/null @@ -1,159 +0,0 @@ - - - - - - PutHDFS - - - - - - -

Description:

-

- This processor writes FlowFiles to an HDFS cluster. It will create directories in which to store files as - needed based on the Directory property. -

- -

- When files are written to HDFS, the file's owner is the user identity of the NiFi process, the file's group is the - group of the parent directory, and the read/write/execute permissions use the default umask. The owner can be - overridden using the Remote Owner property, the group can be overridden using the Remote Group - property, and the read/write/execute permissions can be overridden using the Permissions umask property. -

-

NOTE: This processor can change owner or group only if the user identity of the NiFi process has super user - privilege in HDFS to do so.

-

- NOTE: The Permissions umask property cannot add execute permissions to regular files. -

- -

- Uses Attributes: -

- - - - - - - - - - - - - -
Attribute NameDescription
filenameThe name of the file written to HDFS comes from the value of this attribute.
- -

Properties:

-

In the list below, the names of required properties appear in bold. Any other properties (not in bold) are - considered optional. If a property has a default value, it is indicated. If a property supports the use of the - NiFi Expression Language (or simply, "expression language"), that is also indicated.

-
    -
  • Hadoop Configuration Resources -
      -
    • A file or comma separated list of files which contains the Hadoop file system configuration. - Without this, Hadoop will search the classpath for a 'core-site.xml' and 'hdfs-site.xml' file or will - revert to a default configuration.
    • -
    • Default value: none
    • -
    -
  • -
  • Directory -
      -
    • The HDFS directory to which FlowFile content should be written. This property supports the - expression language so you can keep the FlowFile's directory structure by using the ${path} attribute - reference, e.g. /in/data/${path}.
    • -
    • Default value: none
    • -
    • Supports expression language: true
    • -
  • -
  • Conflict Resolution Strategy -
      -
    • Indicates what should happen when a file with the same name already exists in the output directory. - Valid options are:
    • -
        -
      • replace - existing file is overwritten by new file
      • -
      • ignore - existing file is untouched, FlowFile routed to success
      • -
      • fail - existing file is untouched, FlowFile routed to failure
      • -
      -
    • Default value: fail
    • -
  • -
  • Block Size -
      -
    • Size of each block as written to HDFS. This is a data size integer that must include units of B, - KB, MB, GB, or TB. This overrides the Hadoop Configuration.
    • -
    • Default value: none
    • -
    -
  • -
  • IO Buffer Size -
      -
    • Amount of memory to use to buffer file contents during IO. This is a data size integer that must - include units of B, KB, MB, GB, or TB. This overrides the Hadoop Configuration.
    • -
    • Default value: none
    • -
    -
  • -
  • Replication -
      -
    • Number of times that HDFS will replicate each file. This must be an integer greater than 0. This - overrides the Hadoop Configuration.
    • -
    • Default value: none
    • -
    -
  • -
  • Permissions umask -
      -
    • A umask represented as an octal number which determines the permissions of files written to HDFS. - This overrides the Hadoop Configuration dfs.umaskmode.
    • -
    • Default value: none
    • -
    -
  • -
  • Remote Owner -
      -
    • Changes the owner of the HDFS file to this value after it is written. This only works if NiFi is - running as a user that has HDFS super user privilege to change owner.
    • -
    • Default value: none
    • -
    -
  • -
  • Remote Group -
      -
    • Changes the group of the HDFS file to this value after it is written. This only works if NiFi is - running as a user that has HDFS super user privilege to change group.
    • -
    • Default value: none
    • -
    -
  • -
- -

Relationships:

-
    -
  • success -
      -
    • Files that have been successfully written to HDFS are transferred to this relationship.
    • -
    -
  • -
  • failure -
      -
    • Files that could not be written to HDFS for some reason are transferred to this relationship.
    • -
    -
  • -
- -

- See Also: -

- - - diff --git a/nifi/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/resources/docs/org.apache.nifi.processors.kafka.GetKafka/additionalDetails.html b/nifi/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/resources/docs/org.apache.nifi.processors.kafka.GetKafka/additionalDetails.html new file mode 100644 index 0000000000..daee6ee711 --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/resources/docs/org.apache.nifi.processors.kafka.GetKafka/additionalDetails.html @@ -0,0 +1,78 @@ + + + + + + GetKafka + + + + + +

Description:

+

+ This Processors polls Apache Kafka + for data. When a message is received from Kafka, this Processor emits a FlowFile + where the content of the FlowFile is the value of the Kafka message. If the + message has a key associated with it, an attribute named kafka.key + will be added to the FlowFile, with the value being the UTF-8 Encoded value + of the Message's Key. +

+

+ Kafka supports the notion of a Consumer Group when pulling messages in order to + provide scalability while still offering a publish-subscribe interface. Each + Consumer Group must have a unique identifier. The Consumer Group identifier that + is used by NiFi is the UUID of the Processor. This means that all of the nodes + within a cluster will use the same Consumer Group Identifier so that they do + not receive duplicate data but multiple GetKafka Processors can be used to pull + from multiple Topics, as each Processor will receive a different Processor UUID + and therefore a different Consumer Group Identifier. +

+ +

+ Modifies Attributes: +

+ + + + + + + + + + + + + + + + + + + + + + + + + +
Attribute NameDescription
kafka.topicThe name of the Kafka Topic from which the message was received
kafka.keyThe key of the Kafka message, if it exists and batch size is 1. If the message does not have a key, + or if the batch size is greater than 1, this attribute will not be added.
kafka.partitionThe partition of the Kafka Topic from which the message was received. This attribute is added only + if the batch size is 1.
kafka.offsetThe offset of the message within the Kafka partition. This attribute is added only + if the batch size is 1.
+ + diff --git a/nifi/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/resources/docs/org.apache.nifi.processors.kafka.GetKafka/index.html b/nifi/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/resources/docs/org.apache.nifi.processors.kafka.GetKafka/index.html deleted file mode 100644 index 279dd759f4..0000000000 --- a/nifi/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/resources/docs/org.apache.nifi.processors.kafka.GetKafka/index.html +++ /dev/null @@ -1,173 +0,0 @@ - - - - - - GetKafka - - - - - -

Description:

-

- This Processors polls Apache Kafka - for data. When a message is received from Kafka, this Processor emits a FlowFile - where the content of the FlowFile is the value of the Kafka message. If the - message has a key associated with it, an attribute named kafka.key - will be added to the FlowFile, with the value being the UTF-8 Encoded value - of the Message's Key. -

-

- Kafka supports the notion of a Consumer Group when pulling messages in order to - provide scalability while still offering a publish-subscribe interface. Each - Consumer Group must have a unique identifier. The Consumer Group identifier that - is used by NiFi is the UUID of the Processor. This means that all of the nodes - within a cluster will use the same Consumer Group Identifier so that they do - not receive duplicate data but multiple GetKafka Processors can be used to pull - from multiple Topics, as each Processor will receive a different Processor UUID - and therefore a different Consumer Group Identifier. -

- -

- Modifies Attributes: -

- - - - - - - - - - - - - - - - - - - - - - - - - -
Attribute NameDescription
kafka.topicThe name of the Kafka Topic from which the message was received
kafka.keyThe key of the Kafka message, if it exists and batch size is 1. If the message does not have a key, - or if the batch size is greater than 1, this attribute will not be added.
kafka.partitionThe partition of the Kafka Topic from which the message was received. This attribute is added only - if the batch size is 1.
kafka.offsetThe offset of the message within the Kafka partition. This attribute is added only - if the batch size is 1.
- - -

- Properties: -

-

In the list below, the names of required properties appear - in bold. Any other properties (not in bold) are considered optional. - If a property has a default value, it is indicated. If a property - supports the use of the NiFi Expression Language (or simply, - "expression language"), that is also indicated.

-
    -
  • ZooKeeper Connection String -
      -
    • The Connection String to use in order to connect to ZooKeeper. This is often a - comma-separated list of <host>:<port> combinations. For example, - host1:2181,host2:2181,host3:2188
    • -
    • Default value: no default
    • -
    • Supports expression language: false
    • -
    -
  • -
  • Topic Name -
      -
    • The Kafka Topic to pull messages from
    • -
    • Default value: no default
    • -
    • Supports expression language: false
    • -
    -
  • -
  • Zookeeper Commit Frequency -
      -
    • Specifies how often to communicate with ZooKeeper to indicate which messages have been pulled. - A longer time period will result in better overall performance but can result in more data - duplication if a NiFi node is lost -
    • -
    • Default value: 60 secs
    • -
    • Supports expression language: false
    • -
    -
  • -
  • ZooKeeper Communications Timeout -
      -
    • The amount of time to wait for a response from ZooKeeper before determining that there is a communications error
    • -
    • Default value: 30 secs
    • -
    • Supports expression language: false
    • -
    -
  • -
  • Kafka Communications Timeout -
      -
    • The amount of time to wait for a response from Kafka before determining that there is a communications error
    • -
    • Default value: 30 secs
    • -
    • Supports expression language: false
    • -
    -
  • - -
  • Batch Size -
      -
    • Specifies the maximum number of messages to combine into a single FlowFile. - These messages will be concatenated together with the <Message Demarcator> - string placed between the content of each message. If the messages from Kafka - should not be concatenated together, leave this value at 1.
    • -
    • Default value: 1
    • -
    • Supports expression language: false
    • -
    -
  • - -
  • Message Demarcator -
      -
    • Specifies the characters to use in order to demarcate multiple messages from Kafka. - If the <Batch Size> property is set to 1, this value is ignored. Otherwise, for each two - subsequent messages in the batch, this value will be placed in between them. This property will - treat "\n" as a new-line, "\r" as a carriage return and "\t" as a tab character. All other - characters are treated as literal characters. -
    • -
    • Default value: \n
    • -
    • Supports expression language: false
    • -
    -
  • -
  • Client Name -
      -
    • Client Name to use when communicating with Kafka
    • -
    • Default value: "NiFi-" followed by the UUID of the Processor
    • -
    • Supports expression language: false
    • -
    -
  • - -
-

- Relationships: -

-
    -
  • success -
      -
    • All messages that are received from Kafka are routed to the 'success' relationship
    • -
    -
  • -
- - - diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.ValidateXml/index.html b/nifi/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/resources/docs/org.apache.nifi.processors.kafka.PutKafka/additionalDetails.html similarity index 50% rename from nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.ValidateXml/index.html rename to nifi/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/resources/docs/org.apache.nifi.processors.kafka.PutKafka/additionalDetails.html index d4db3cb4aa..04d9463ed6 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.ValidateXml/index.html +++ b/nifi/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/resources/docs/org.apache.nifi.processors.kafka.PutKafka/additionalDetails.html @@ -16,8 +16,7 @@ --> - ValidateXml - + PutKafka @@ -25,32 +24,22 @@

Description:

- This processor validates the contents of FlowFiles against a user-specified XML schema file. -

- - Properties: + This Processors puts the contents of a FlowFile to a Topic in + Apache Kafka. The full contents of + a FlowFile becomes the contents of a single message in Kafka. + This message is optionally assigned a key by using the + <Kafka Key> Property.

-

- In the list below, the names of required properties appear in bold. Any other properties (not in bold) are considered optional. If a property has a default value, it is indicated. If a property supports the use of the NiFi Expression Language (or simply, "expression language"), that is also indicated. -

-
    -
  • Schema File -
      -
    • The local file path to the schema file to be used for validation.
    • -
    • Default value: no default
    • -
    • Supports expression language: false
    • -
  • -
- -

- Relationships: -

-
    -
  • valid -
      -
    • If FlowFiles are successfully validated against the schema, then they follow this relationship.
    • -
  • -
+

+ The Processor allows the user to configure an optional Message Delimiter that + can be used to send many messages per FlowFile. For example, a \n could be used + to indicate that the contents of the FlowFile should be used to send one message + per line of text. If the property is not set, the entire contents of the FlowFile + will be sent as a single message. When using the delimiter, if some messages are + successfully sent but other messages fail to send, the FlowFile will be FORKed into + two child FlowFiles, with the successfully sent messages being routed to 'success' + and the messages that could not be sent going to 'failure'. +

diff --git a/nifi/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/resources/docs/org.apache.nifi.processors.kafka.PutKafka/index.html b/nifi/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/resources/docs/org.apache.nifi.processors.kafka.PutKafka/index.html deleted file mode 100644 index 29b7c176f5..0000000000 --- a/nifi/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/resources/docs/org.apache.nifi.processors.kafka.PutKafka/index.html +++ /dev/null @@ -1,189 +0,0 @@ - - - - - - PutKafka - - - - - -

Description:

-

- This Processors puts the contents of a FlowFile to a Topic in - Apache Kafka. The full contents of - a FlowFile becomes the contents of a single message in Kafka. - This message is optionally assigned a key by using the - <Kafka Key> Property. -

- -

- The Processor allows the user to configure an optional Message Delimiter that - can be used to send many messages per FlowFile. For example, a \n could be used - to indicate that the contents of the FlowFile should be used to send one message - per line of text. If the property is not set, the entire contents of the FlowFile - will be sent as a single message. When using the delimiter, if some messages are - successfully sent but other messages fail to send, the FlowFile will be FORKed into - two child FlowFiles, with the successfully sent messages being routed to 'success' - and the messages that could not be sent going to 'failure'. -

- -

- Properties: -

-

In the list below, the names of required properties appear - in bold. Any other properties (not in bold) are considered optional. - If a property has a default value, it is indicated. If a property - supports the use of the NiFi Expression Language (or simply, - "expression language"), that is also indicated.

-
    -
  • Known Brokers -
      -
    • - A comma-separated list of known Kafka Brokers in the format - <host>:<port>. This list does not need to be - exhaustive but provides a mechanism for determining which - other nodes belong to the Kafka cluster. -
    • -
    • Default value: no default
    • -
    • Supports expression language: false
    • -
    -
  • -
  • Topic Name -
      -
    • The Kafka Topic to send messages to. While the GetKafka - Processor requires a statically named Topic so that it knows - where to fetch messages from, the PutKafka Processor does allow - the Expression Language to be used so that a single PutKafka - Processor can be used to send messages to many different Kafka - topics. -
    • -
    • Default value: no default
    • -
    • Supports expression language: true
    • -
    -
  • - -
  • Kafka Key -
      -
    • - The Key to use for the Message. If no value is given, the message - will not be given a Key. -
    • -
    • Default value: no default
    • -
    • Supports expression language: true
    • -
    -
  • -
  • Delivery Guarantee -
      -
    • - Specifies the requirement for guaranteeing that a message is sent to Kafka. - This Property can have one of three different values: -
        -
      • - Guarantee Replicated Delivery - FlowFile will be routed to - failure unless the message is replicated to the appropriate number - of Kafka Nodes according to the Topic configuration -
      • -
      • - Guarantee Single Node Delivery - FlowFile will be routed to - success if the message is received by a single Kafka node, - whether or not it is replicated. This is faster than - <Guarantee Replicated Delivery> but can result in data loss - if a Kafka node crashes -
      • -
      • - Best Effort - FlowFile will be routed to success after successfully - writing the content to a Kafka node, without waiting for a response. - This provides the best performance but may result in data loss. -
      • -
      -
    • -
    • Default value: Best Effort
    • -
    • Supports expression language: false
    • -
    -
  • -
  • Message Delimiter -
      -
    • - Specifies the delimiter to use for splitting apart multiple messages within a single FlowFile. - If not specified, the entire content of the FlowFile will be used as a single message. - If specified, the contents of the FlowFile will be split on this delimiter and each section - sent as a separate Kafka message. -
    • -
    • Default value: no default
    • -
    • Supports expression language: true
    • -
    -
  • -
  • Communications Timeout -
      -
    • - The amount of time to wait for a response from Kafka before determining - that there is a communications error -
    • -
    • Default value: 30 secs
    • -
    • Supports expression language: false
    • -
    -
  • -
  • Max Buffer Size -
      -
    • - The maximum amount of data to buffer in memory before sending to Kafka -
    • -
    • Default value: 1 MB
    • -
    • Supports expression language: false
    • -
    -
  • -
  • Client Name -
      -
    • Client Name to use when communicating with Kafka
    • -
    • Default value: "NiFi-" followed by the UUID of the Processor
    • -
    • Supports expression language: false
    • -
    -
  • -
- - -

- Relationships: -

-
    -
  • success -
      -
    • All FlowFiles that are successfully sent to Kafka are routed - to this relationship. If using the <Message Delimiter> property, - it's possible for some messages to be sent while others fail. In this - case, only the messages that are successfully sent will be routed to - this Relationship while the other messages will be routed to the - 'failure' relationship. -
    • -
    -
  • - -
  • failure -
      -
    • All FlowFiles that cannot be sent to Kafka for any reason be routed - to this relationship. If a portion of a FlowFile is successfully sent - to Kafka but not all, only those messages that cannot be sent to Kafka - will be routed to this Relationship. -
    • -
    -
  • - -
- - - diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.Base64EncodeContent/index.html b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.Base64EncodeContent/index.html deleted file mode 100644 index c6a4b4d3ba..0000000000 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.Base64EncodeContent/index.html +++ /dev/null @@ -1,63 +0,0 @@ - - - - - - EncodeContent - - - - - -

Description:

-

- This processor base64 encodes FlowFile content, or decodes FlowFile content from base64. -

- Properties: -

-

- In the list below, the names of required properties appear in bold. Any other properties (not in bold) are considered optional. If a property has a default value, it is indicated. If a property supports the use of the NiFi Expression Language (or simply, "expression language"), that is also indicated. -

- -
    -
  • Mode -
      -
    • This property specifies whether the content should be base64 encoded, or decoded from base64. Valid values are: -
        -
      • Encode
      • -
      • Decode
      • -
    • -
    • Default value: Encode
    • -
    • Supports expression language: false
    • -
  • -
-

- Relationships: -

-
    -
  • failure -
      -
    • If a FlowFile cannot be encoded or decoded, then it follows this relationship
    • -
  • -
  • success -
      -
    • If a FlowFile is successfully encoded or decoded, then it follows this relationship
    • -
  • -
- - - - diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.CompressContent/additionalDetails.html b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.CompressContent/additionalDetails.html new file mode 100644 index 0000000000..4339e32c05 --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.CompressContent/additionalDetails.html @@ -0,0 +1,68 @@ + + + + + + CompressContent + + + + + + + + +

+ Uses Attributes: +

+ + + + + + + + + + + + + +
Attribute NameDescription
mime.typeIf the Compression Format is set to use mime.type attribute, + this attribute is used to determine the compression type. Otherwise, this attribute is ignored.
+ +

+ Modifies Attributes: +

+ + + + + + + + + + + + + +
Attribute NameDescription
mime.typeIf the Mode property is set to compress, the appropriate MIME Type is set. If the + Mode property is set to decompress and the file is successfully decompressed, + this attribute is removed, as the MIME Type is no longer known. +
+ + diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.CompressContent/index.html b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.CompressContent/index.html deleted file mode 100644 index aa5251ded5..0000000000 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.CompressContent/index.html +++ /dev/null @@ -1,166 +0,0 @@ - - - - - - CompressContent - - - - - - - -

Description:

-

-

- This processor compresses and decompresses the contents of - FlowFiles using a user-specified compression algorithm. -

- -

- Uses Attributes: -

- - - - - - - - - - - - - -
Attribute NameDescription
mime.typeIf the Compression Format is set to use mime.type attribute, - this attribute is used to determine the compression type. Otherwise, this attribute is ignored.
- -

- Modifies Attributes: -

- - - - - - - - - - - - - -
Attribute NameDescription
mime.typeIf the Mode property is set to compress, the appropriate MIME Type is set. If the - Mode property is set to decompress and the file is successfully decompressed, - this attribute is removed, as the MIME Type is no longer known. -
- -

- Properties: -

-

- In the list below, the names of required properties appear in bold. - Any other properties (not in bold) are considered optional. If a - property has a default value, it is indicated. If a property - supports the use of the NiFi Expression Language (or simply, - "expression language"), that is also indicated. -

- -
    -
  • - Mode -
      -
    • - Indicates whether the processor should compress or decompress - content. Valid values are: -
        -
      • compress
      • -
      • decompress
      • -
      -
    • -
    • Default value: compress
    • -
    • Supports expression language: false
    • -
    -
  • -
  • - Compression Format -
      -
    • - The compression format to use. Valid values are: -
        -
      • gzip : GZIP compression format
      • -
      • bzip2 : BZIP2 compression format
      • -
      • xz-lzma2 : XZ-LZMA2 compression format
      • -
      • lzma : LZMA compression format
      • -
      • - use mime.type attribute : The compression format should be - determined by examining the - mime.type - attribute. - If the attribute is not present, the FlowFile will be routed to - 'failure'. If the attribute is present but does not denote - one of the supported compression formats, the FlowFile will be - routed to 'success' without being modified. This allows - CompressContent - to follow the - IdentifiyMimeType Processor - so that files that are - compressed can automatically be decompressed and files that are not - compressed will continue on. -
      • -
      -
    • -
    • Default value: use mime.type attribute
    • -
    • Supports expression language: false
    • -
    -
  • -
  • - Compression Level -
      -
    • The compression level to use. This is valid only when using - the gzip compression format. A lower value results in faster - processing but less compression. A value of zero (0) indicates no - compression but simply archiving. Valid values are 0-9.
    • -
    • Default value: 1
    • -
    • Supports expression language: false.
    • -
    -
  • -
-

- Relationships: -

-
    -
  • - failure -
      -
    • If FlowFiles fail to compress or decompress, then they follow - this relationship.
    • -
    -
  • -
  • - success -
      -
    • If FlowFiles are successfully compressed or decompressed, then - they follow this relationship.
    • -
    -
  • -
- - - diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.ControlRate/index.html b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.ControlRate/index.html deleted file mode 100644 index cbe1088ad2..0000000000 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.ControlRate/index.html +++ /dev/null @@ -1,116 +0,0 @@ - - - - - - ControlRate - - - - - - - -

Description:

-

This processor controls the rate at which data is - transferred to follow-on processors. -

- -

- Properties: -

-

In the list below, the names of required properties appear - in bold. Any other properties (not in bold) are considered optional. - If a property has a default value, it is indicated. If a property - supports the use of the NiFi Expression Language (or simply, - "expression language"), that is also indicated.

- -
    -
  • Rate Control Criteria -
      -
    • Indicates the criteria used to control the throughput rate. - Changing this value resets the rate counters. Possible values are: -
        -
      • data rate
      • -
      • flowfile count
      • -
      • attribute value
      • -
      -
    • -
    • Default value: data rate
    • -
    • Supports expression language: false
    • -
  • -
  • Maximum Rate -
      -
    • The maximum rate at which data should pass through this - processor. The format of this value is expected to be a - non-negative integer or, if the Rate Control Criteria is set to - data rate, a data size (such as 1MB).
    • -
    • Default value: no default
    • -
    • Supports expression language: false
    • -
  • -
  • Rate Controlled Attribute -
      -
    • The name of an attribute whose values build toward the rate - limit if the Rate Control Criteria is set to attribute value. The - value of the attribute referenced by this property must be a - non-negative integer; otherwise, the flowfile will be routed to the - failure relationship. This value is ignored if the Rate Control - Criteria is not set to attribute value. Changing this value resets - the rate counters.
    • -
    • Default value: no default
    • -
    • Supports expression language: true.
    • -
    -
  • -
  • Time Duration -
      -
    • The amount of time to which the Maximum Rate property - pertains. Changing this value resets the rate counters.
    • -
    • Default value: 1 min
    • -
    • Supports expression language: false.
    • -
  • -
  • Grouping Attribute -
      -
    • By default, a single "throttle" is used for all FlowFiles. - If this value is specified, a separate throttle is used for each - value specified by the attribute with this name. Changing this - value resets the rate counters.
    • -
    • Default value: no default
    • -
    • Supports expression language: true.
    • -
    -
  • -
-

- Relationships: -

-
    -
  • success -
      -
    • If FlowFiles are successfully processed, then they follow - this relationship.
    • -
    -
  • -
- - - - - - - - - - - diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.ConvertCharacterSet/index.html b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.ConvertCharacterSet/index.html deleted file mode 100644 index 9a888151e4..0000000000 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.ConvertCharacterSet/index.html +++ /dev/null @@ -1,65 +0,0 @@ - - - - - - ConvertCharacterSet - - - - - -

Description:

-

This processor converts a FlowFile's content from one - character set to another.

-

- Properties: -

-

In the list below, the names of required properties appear - in bold. Any other properties (not in bold) are considered optional. - If a property has a default value, it is indicated. If a property - supports the use of the NiFi Expression Language (or simply, - "expression language"), that is also indicated.

- -
    -
  • Input Character Set -
      -
    • The name of the character set to expect for the input.
    • -
    • Default value: no default
    • -
    • Supports expression language: false
    • -
  • -
  • Output Character Set -
      -
    • The name of the character set to convert to
    • -
    • Default value: no default
    • -
    • Supports expression language: false
    • -
  • -
-

- Relationships: -

-
    -
  • success -
      -
    • If a FlowFile's character set is successfully converted, - then it follows this relationship.
    • -
    -
  • -
- - - - diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.DetectDuplicate/additionalDetails.html b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.DetectDuplicate/additionalDetails.html new file mode 100644 index 0000000000..c0b65b9167 --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.DetectDuplicate/additionalDetails.html @@ -0,0 +1,58 @@ + + + + + + DetectDuplicate + + + + + + +

+ Modifies Attributes: +

+ + + + + + + + + + + + + +
Attribute NameDescription
original.flowfile.descriptionAll FlowFiles routed to the duplicate + relationship will have an attribute added named original.flowfile.description. + The value of this attribute is determined by the attributes of the + original copy of the data and by the FlowFile Description property +
+ +

+ See Also: +

+ + + + + diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.DetectDuplicate/index.html b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.DetectDuplicate/index.html deleted file mode 100644 index 87e066c91d..0000000000 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.DetectDuplicate/index.html +++ /dev/null @@ -1,147 +0,0 @@ - - - - - - DetectDuplicate - - - - - -

Description:

-

This processor detects duplicate data by examining flow file attributes, - thus allowing the user to configure what it means for two FlowFiles to be - considered "duplicates". This processor does not read the contents of a - flow file, and is typically preceded by another processor which computes a - value based on the flow file content and adds that value to the flow file's - attributes; e.g. HashContent. Because this Processor needs to be able - to work within a NiFi cluster, it makes use of a distributed cache service to - determine whether or not the data has been seen previously.

- -

- If the processor is to be run on a standalone instance of - NiFi, that instance should have both a - DistributedMapCacheClient and a DistributedMapCacheServer configured - in its controller-services.xml file. -

- -

- Modifies Attributes: -

- - - - - - - - - - - - - -
Attribute NameDescription
original.flowfile.descriptionAll FlowFiles routed to the duplicate - relationship will have an attribute added named original.flowfile.description. - The value of this attribute is determined by the attributes of the - original copy of the data and by the FlowFile Description property -
- -

- Properties: -

-

In the list below, the names of required properties appear - in bold. Any other properties (not in bold) are considered optional. - If a property has a default value, it is indicated. If a property - supports the use of the NiFi Expression Language (or simply, - "expression language"), that is also indicated.

- -
    -
  • Distributed Cache Service -
      -
    • The Controller Service that is used to cache unique - identifiers which are used to determine duplicates
    • -
    • Default value: (None)
    • -
    • Supports expression language: false
    • -
  • -
  • Cache Entry Identifier -
      -
    • A FlowFile attribute, or the results of an Attribute Expression Language statement, - which will be evaluated against a FlowFile in order to determine the value used to - identify duplicates; it is this value that is cached
    • -
    • Default value: ${hash.value}
    • -
    • Supports expression language: true
    • -
  • -
  • FlowFile Description -
      -
    • When a FlowFile is added to the cache, this value is stored - along with it so that if a duplicate is found, this description of - the original FlowFile will be added to the duplicate's - "original.flowfile.description" attribute
    • -
    • Default value:
    • -
    • Supports expression language: true
    • -
  • -
  • Age Off Duration -
      -
    • Time interval to age off FlowFiles from the Distributed Cache Service; must specify time unit
    • -
    • Default value: none
    • -
    • Supports expression language: false
    • -
    -
  • -
-

- Relationships: -

-
    -
  • duplicate -
      -
    • If a FlowFile has been detected to be a duplicate, it will - be routed to this relationship. This FlowFile will have an - attribute named "original.flowfile.description" that provides - details about the original version of the FlowFile.
    • -
    -
  • -
  • non-duplicate -
      -
    • If a FlowFile's Cache Entry Identifier was not found in the - cache, it will be routed to this relationship. In this case, the - FlowFile's "description" (as defined by the FlowFile - Description property) will be stored in the cache. -
    • -
    -
  • -
  • failure -
      -
    • If unable to communicate with the cache or if the FlowFile does not - have the attribute identified by the Cache Entry Identifier, the FlowFile will - be penalized and routed to this relationship
    • -
    -
  • -
- - -

- See Also: -

- - - - - diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.DistributeLoad/index.html b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.DistributeLoad/additionalDetails.html similarity index 54% rename from nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.DistributeLoad/index.html rename to nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.DistributeLoad/additionalDetails.html index f1ca056d68..0c16e4fc7f 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.DistributeLoad/index.html +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.DistributeLoad/additionalDetails.html @@ -34,58 +34,12 @@ FlowFiles in each iteration instead of 1. All other relationships will receive 1 FlowFile in each iteration.

Properties: -

In the list below, the names of required properties appear in bold. Any other properties (not in bold) are considered optional. If a property has a default value, it is indicated. If a property supports the use of the NiFi Expression Language (or simply, "expression language"), that is also indicated.

-
    -
  • Number of Relationships -
      -
    • The number of relationships to which the load should be - distributed. A valid value is a non-negative integer.
    • -
    • Default value: 1
    • -
    • Supports expression language: false
    • -
  • -
  • Distribution Strategy -
      -
    • This property determines how the load is distributed. If - the round robin strategy is used, the processor does not distribute - any FlowFiles unless all destinations can accept FlowFiles. If the - next available strategy is used, the processor distributes - FlowFiles as long as at least one destination can accept FlowFiles. - If the load distribution service is used, the processor ignores any - user specified optional properties, requires the service id of the - Load Distribution Service, and requires a list of fully qualified host - names, or FQDN's, which are the distribution end points. - Valid values are: -
        -
      • round robin
      • -
      • next available
      • -
      • load distribution service
      • -
      -
    • -
    • Default value: round robin
    • -
    • Supports expression language: false
    • -
    -
  • -
  • Hostnames - this is required when using 'load distribution service' strategy -
      -
    • List of remote servers to distribute across. Each server must be FQDN and use either ',', ';', or 'space' as a delimiter
    • -
    • Default Value: none
    • -
    • Supports expression language: false
    • -
    -
  • -
  • Load Distribution Service ID - this is required when using 'load distribution service' strategy -
      -
    • The identifier of the Load Distribution Service
    • -
    • Default Value: none
    • -
    • Supports expression language: false
    • -
    -
  • -

Relationships:

diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.EncryptContent/index.html b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.EncryptContent/index.html deleted file mode 100644 index ed49aaa80b..0000000000 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.EncryptContent/index.html +++ /dev/null @@ -1,97 +0,0 @@ - - - - - - EncryptContent - - - - - -

Description:

-

- This processor encrypts or decrypts FlowFiles. -

- Properties: -

-

- In the list below, the names of required properties appear in bold. Any other properties (not in bold) are considered optional. If a property has a default value, it is indicated. If a property supports the use of the NiFi Expression Language (or simply, "expression language"), that is also indicated. -

- -
    -
  • Mode -
      -
    • This property specifies whether the content should be encrypted or decrypted. Valid values are: -
        -
      • Encrypt
      • -
      • Decrypt
      • -
    • -
    • Default value: Encrypt
    • -
    • Supports expression language: false
    • -
  • -
  • Encryption Algorithm -
      -
    • The encryption algorithm to use. Valid values are available in the property's drop down menu; they include: -
        -
      • MD5_128AES - requires only 'strong encryption policies'
      • -
      • MD5_256AES - requires only 'strong encryption policies'
      • -
      • SHA1_RC2 - requires only 'strong encryption policies'
      • -
      • SHA1_DES - requires only 'strong encryption policies'
      • -
      • MD5_192AES - requires only 'strong encryption policies'
      • -
      • MD5_DES - requires only 'strong encryption policies'
      • -
      • MD5_RC2 - requires only 'strong encryption policies'
      • -
      • SHA_192AES - requires unlimited JCE policies
      • -
      • SHA_40RC4 - requires unlimited JCE policies
      • -
      • SHA256_128AES - requires unlimited JCE policies
      • -
      • SHA_128RC2 - requires unlimited JCE policies
      • -
      • SHA_128AES - requires unlimited JCE policies
      • -
      • SHA256_192AES - requires unlimited JCE policies
      • -
      • SHA_2KEYTRIPLEDES - requires unlimited JCE policies
      • -
      • SHA256_256AES - requires unlimited JCE policies
      • -
      • SHA_40RC2 - requires unlimited JCE policies
      • -
      • SHA_256AES - requires unlimited JCE policies
      • -
      • SHA_3KEYTRIPLEDES - requires unlimited JCE policies
      • -
      • SHA_TWOFISH - requires unlimited JCE policies
      • -
      • SHA_128RC4 - requires unlimited JCE policies
      • -
    • -
    • Default value: MD5_256AES
    • -
    • Supports expression language: false
    • -
  • -
  • Password -
      -
    • The password to use for encrypting or decrypting the data. Passwords longer than 16 characters require unlimited JCE policies
    • -
    • Default value: no default
    • -
    • Supports expression language: false
    • -
  • -
-

- Relationships: -

-
    -
  • failure -
      -
    • If a FlowFile cannot be encrypted or decrypted, then it follows this relationship
    • -
  • -
  • success -
      -
    • If a FlowFile is successfully encrypted or decrypted, then it follows this relationship
    • -
  • -
- - - - diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.EvaluateRegularExpression/index.html b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.EvaluateRegularExpression/index.html deleted file mode 100644 index 87a99dbe2f..0000000000 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.EvaluateRegularExpression/index.html +++ /dev/null @@ -1,160 +0,0 @@ - - - - - - EvaluateRegularExpression - - - - - -

Description:

-

- This processor evaluates one or more Regular Expressions against the content of a FlowFile. - The results of those Regular Expressions are assigned to FlowFile Attributes. - Regular Expressions are entered by adding user-defined properties; - the name of the property maps to the Attribute Name into which the result will be placed. - The value of the property must be a valid Regular Expressions with exactly one capturing group. - If the Regular Expression matches more than once, only the first match will be used. - If any provided Regular Expression matches, the FlowFile(s) will be routed to 'matched'. - If no provided Regular Expression matches, the FlowFile will be routed to 'unmatched' and no attributes will be applied to the FlowFile. - -

- Properties: -

-

- In the list below, the names of required properties appear in bold. - Any other properties (not in bold) are considered optional. If a - property has a default value, it is indicated. If a property - supports the use of the NiFi Expression Language (or simply, - "expression language"), that is also indicated. -

- -

- Modifies Attributes: -

-

- This processor adds user-defined attributes. -

- -
    -
  • Character Set -
      -
    • The character set in which the file is encoded.
    • -
    • Default value: UTF-8
    • -
    • Supports expression language: false
    • -
  • -
  • Maximum Buffer Size -
      -
    • Specifies the maximum amount of data to buffer (per file) in order to apply the regular expressions. Files larger than the specified maximum will not be fully evaluated.
    • -
    • Default value: 1MB
    • -
    • Supports expression language: false
    • -
  • - -
  • Enable Canonical Equivalence -
      -
    • Indicates that two characters match only when their full canonical decompositions match.
    • -
    • Default value: false
    • -
    • Supports expression language: false
    • -
  • - -
  • Enable Case-insensitive Matching -
      -
    • Indicates that two characters match even if they are in a different case. Can also be specified via the embeded flag (?i).
    • -
    • Default value: false
    • -
    • Supports expression language: false
    • -
  • - -
  • Permit Whitespace and Comments in Pattern -
      -
    • In this mode, whitespace is ignored, and embedded comments starting with # are ignored until the end of a line. Can also be specified via the embeded flag (?x).
    • -
    • Default value: false
    • -
    • Supports expression language: false
    • -
  • - -
  • Enable DOTALL Mode -
      -
    • Indicates that the expression '.' should match any character, including a line terminator. Can also be specified via the embeded flag (?s).
    • -
    • Default value: false
    • -
    • Supports expression language: false
    • -
  • - -
  • Enable Literal Parsing of the Pattern -
      -
    • Indicates that Metacharacters and escape characters should be given no special meaning.
    • -
    • Default value: false
    • -
    • Supports expression language: false
    • -
  • - -
  • Enable Multiline Mode -
      -
    • Indicates that '^' and '$' should match just after and just before a line terminator or end of sequence, instead of only the begining or end of the entire input. - Can also be specified via the embeded flag (?m).
    • -
    • Default value: false
    • -
    • Supports expression language: false
    • -
  • - -
  • Enable Unicode-aware Case Folding -
      -
    • When used with 'Enable Case-insensitive Matching', matches in a manner consistent with the Unicode Standard. Can also be specified via the embeded flag (?u).
    • -
    • Default value: false
    • -
    • Supports expression language: false
    • -
  • - -
  • Enable Unicode Predefined Character Classes -
      -
    • Specifies conformance with the Unicode Technical Standard #18: Unicode Regular Expression Annex C: Compatibility Properties. Can also be specified via the embeded flag (?U).
    • -
    • Default value: false
    • -
    • Supports expression language: false
    • -
  • - -
  • Enable Unix Lines Mode -
      -
    • Indicates that only the '\n' line terminator is recognized int the behavior of '.', '^', and '$'. Can also be specified via the embeded flag (?d).
    • -
    • Default value: false
    • -
    • Supports expression language: false
    • -
  • - -
  • - user-defined properties -
      -
    • The name of the attribute to put the Regular Expression result into.
    • -
    • Supports expression language: false
    • -
    -
  • -
- -

- Relationships: -

-
    -
  • - matched -
      -
    • FlowFiles are routed to this relationship when the Regular Expression is successfully evaluated and the FlowFile is modified as a result.
    • -
    -
  • -
  • - unmatched -
      -
    • FlowFiles are routed to this relationship when no provided Regular Expression matches the content of the FlowFile.
    • -
    -
  • -
-

- - diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.EvaluateXPath/additionalDetails.html b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.EvaluateXPath/additionalDetails.html new file mode 100644 index 0000000000..5fe58a5b26 --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.EvaluateXPath/additionalDetails.html @@ -0,0 +1,34 @@ + + + + + + EvaluateXPath + + + + + + +

+ Modifies Attributes: +

+

+ This processor adds user-defined attributes if the <Destination> property is set to + flowfile-attribute. +

+ + diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.EvaluateXPath/index.html b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.EvaluateXPath/index.html deleted file mode 100644 index e1f0fa9040..0000000000 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.EvaluateXPath/index.html +++ /dev/null @@ -1,135 +0,0 @@ - - - - - - EvaluateXPath - - - - - -

Description:

-

- This processor evaluates one or more XPaths against the content - of FlowFiles. The results of those XPaths are assigned to FlowFile - attributes or are written to the content of the FlowFile itself, - depending on how the user configures the Destination and Return Type properties in the - processor. XPaths are entered by adding user-defined properties; the - name of each user-added property maps to the attribute name into - which the result should be placed. The value of the property must be - a valid XPath expression. -

- Properties: -

-

- In the list below, the names of required properties appear in bold. - Any other properties (not in bold) are considered optional. If a - property has a default value, it is indicated. If a property - supports the use of the NiFi Expression Language (or simply, - "expression language"), that is also indicated. -

- -

- Modifies Attributes: -

-

- This processor adds user-defined attributes if the <Destination> property is set to - flowfile-attribute. -

- - -
    -
  • - Destination -
      -
    • Indicates whether the results of the XPath evaluation are - written to the FlowFile content or a FlowFile attribute; if using - attribute, the attribute's name must be specified in the value of - the Attribute Name property.
    • -
    • - Valid values are: -
        -
      • flowfile-content
      • -
      • flowfile-attribute
      • -
      -
    • -
    • Default value: flowfile-content
    • -
    • Supports expression language: false
    • -
    -
  • -
  • - Return Type -
      -
    • Indicates the desired return type of the Xpath expressions. - Selecting 'auto-detect' will set the return type to 'nodeset' for a - Destination of 'flowfile-content', and 'string' for a Destination of - 'flowfile-attribute'.")
    • -
    • - Valid values are: -
        -
      • auto-detect
      • -
      • nodeset
      • -
      • string
      • -
      -
    • -
    • Default value: auto-detect
    • -
    • Supports expression language: false
    • -
    -
  • -
  • - user-defined properties -
      -
    • The name of the attribute to put the XPath result into if - flowfile-attribute is used as the value for the Destination - property; if using flowfile-content as the value for the - Destination property, this value is ignored.
    • -
    • Supports expression language: false
    • -
    -
  • -
- -

- Relationships: -

-
    -
  • - failure -
      -
    • If the XPath cannot be evaluated against the content of the - FlowFile, then the FlowFile follows this relationship. For - example, if the FlowFile does not contain valid XML.
    • -
    -
  • -
  • - matched -
      -
    • If the XPath is successfully evaluated and the FlowFile is - modified as a result, then the FlowFile follows this - relationship.
    • -
    -
  • -
  • - unmatched -
      -
    • If the XPath does not match the content of the FlowFile, then - the FlowFile follows this relationship.
    • -
    -
  • -
-

- - diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.EvaluateXQuery/additionalDetails.html b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.EvaluateXQuery/additionalDetails.html new file mode 100644 index 0000000000..3d120a45b7 --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.EvaluateXQuery/additionalDetails.html @@ -0,0 +1,161 @@ + + + + + +EvaluateXQuery + + + + + + + + + +

+ Modifies Attributes: +

+ +

+ This processor adds user-defined attributes if the <Destination> + property is set to + flowfile-attribute + . +

+ + +

+ Examples: +

+ +

This processor produces one attribute or FlowFile per + XQueryResult. If only one attribute or FlowFile is desired, the + following examples demonstrate how this can be achieved using the + XQuery language. The examples below reference the following sample + XML:

+ +
+		
+  <?xml version="1.0" encoding="UTF-8"?>
+  <?xml-stylesheet type="text/xsl" href="foo.xsl"?>
+  <ns:fruitbasket xmlns:ns="http://namespace/1">
+    <fruit taste="crisp">    
+      <!-- Apples are my favorite-->   
+      <name>apple</name>   
+      <color>red</color>  
+    </fruit>  
+    <fruit>   
+      <name>apple</name>   
+      <color>green</color>  
+    </fruit>  
+    <fruit>   
+      <name>banana</name>   
+      <color>yellow</color>  
+    </fruit>  
+    <fruit taste="sweet">   
+      <name>orange</name>   
+      <color>orange</color>  
+    </fruit>  
+    <fruit>   
+      <name>blueberry</name>   
+      <color>blue</color>  
+    </fruit>  
+      <fruit taste="tart">   
+      <name>raspberry</name>   
+      <color>red</color>  
+    </fruit>  
+    <fruit>   
+      <name>none</name>    
+      <color/>  
+    </fruit>
+  </ns:fruitbasket>
+
+	
+ +

+

    +
  • XQuery to return all "fruit" nodes individually (7 Results): +
      +
    • //fruit
    • +
    +
  • +
  • XQuery to return only the first "fruit" node (1 Result): +
      +
    • //fruit[1]
    • +
    +
  • +
  • XQuery to return only the last "fruit" node (1 Result): +
      +
    • //fruit[count(//fruit)]
    • +
    +
  • +
  • XQuery to return all "fruit" nodes, wrapped in a "basket" tag + (1 Result): +
      +
    • <basket>{//fruit}</basket>
    • +
    +
  • +
  • XQuery to return all "fruit" names individually (7 Results): +
      +
    • //fruit/text()
    • +
    +
  • +
  • XQuery to return only the first "fruit" name (1 Result): +
      +
    • //fruit[1]/text()
    • +
    +
  • +
  • XQuery to return only the last "fruit" name (1 Result): +
      +
    • //fruit[count(//fruit)]/text()
    • +
    +
  • +
  • XQuery to return all "fruit" names as a comma separated list + (1 Result): +
      +
    • string-join((for $x in //fruit return $x/name/text()), ', + ')
    • +
    +
  • +
  • XQuery to return all "fruit" colors and names as a comma + separated list (1 Result): +
      +
    • string-join((for $y in (for $x in //fruit return + string-join(($x/color/text() , $x/name/text()), ' ')) return $y), + ', ')
    • +
    +
  • +
  • XQuery to return all "fruit" colors and names as a comma + separated list (1 Result): +
      +
    • string-join((for $y in (for $x in //fruit return + string-join(($x/color/text() , $x/name/text()), ' ')) return $y), + ', ')
    • +
    +
  • +
  • XQuery to return all "fruit" colors and names as a new line + separated list (1 Result): +
      +
    • string-join((for $y in (for $x in //fruit return + string-join(($x/color/text() , $x/name/text()), ' ')) return $y), + '\n')
    • +
    +
  • +
+ + diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.EvaluateXQuery/index.html b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.EvaluateXQuery/index.html deleted file mode 100644 index 7db8cba34b..0000000000 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.EvaluateXQuery/index.html +++ /dev/null @@ -1,311 +0,0 @@ - - - - - - EvaluateXQuery - - - - - - - -

Description:

- -

This processor evaluates one or more XQueries against the - content of FlowFiles. The results of those XQueries are assigned to - FlowFile attributes or are written to the content of the FlowFile - itself, depending on how the user configures the Destination property - in the processor. One attribute or FlowFile is produced for each XQuery - result. Each produced FlowFile will carry the attributes of the input - FlowFile. See the "Examples" section for details on how - multiple results can be wrapped or concatenated. XQueries are - entered by adding user-defined - properties; the name of each user-added property maps to the attribute - name into which the result should be placed. The value of the property - must be a valid XQuery expression.

- -

Properties:

- -

In the list below, the names of required properties - appear in bold. Any other properties (not in bold) are considered - optional. If a property has a default value, it is indicated. If a - property supports the use of the NiFi Expression Language (or - simply, "expression language"), that is also indicated.

- -

Modifies Attributes:

- -

This processor adds user-defined attributes if the - <Destination> property is set to flowfile-attribute - .

- -
    - -
  • Destination -
      - -
    • Indicates whether the results of the XQuery evaluation - are written to the FlowFile content or a FlowFile attribute; if using - attribute, the attribute's name must be specified in the value of the - Attribute Name property.
    • - -
    • Valid values are: -
        - -
      • flowfile-content
      • - -
      • flowfile-attribute
      • - -
      - -
    • - -
    • Default value: flowfile-content
    • - -
    • Supports expression language: false
    • - -
    - -
  • - -
  • Output: Method -
      - -
    • Identifies the overall method that should be used for - outputting a result tree. This property will be ignored if the result - of the XQuery is not a DOCUMENT or ELEMENT Node.
    • - -
    • Valid values are: -
        - -
      • xml
      • - -
      • html
      • - -
      • text
      • - -
      - -
    • - -
    • Default value: xml
    • - -
    • Supports expression language: false
    • - -
    - -
  • - -
  • Output: Omit XML Declaration -
      - -
    • Specifies whether the processor should output an XML - declaration when transforming a result tree. This property will be - ignored if the result of the XQuery is not a DOCUMENT or ELEMENT Node.
    • - -
    • Valid values are: -
        - -
      • true
      • - -
      • false
      • - -
      - -
    • - -
    • Default value: false
    • - -
    • Supports expression language: false
    • - -
    - -
  • - -
  • Output: Indent -
      - -
    • Specifies whether the processor may add additional - whitespace when outputting a result tree. This property will be ignored - if the result of the XQuery is not a DOCUMENT or ELEMENT Node.
    • - -
    • Valid values are: -
        - -
      • true
      • - -
      • false
      • - -
      - -
    • - -
    • Default value: false
    • - -
    • Supports expression language: false
    • - -
    - -
  • - -
  • user-defined properties -
      - -
    • The name of the attribute to put the XQuery result into - if flowfile-attribute is used as the value for the Destination - property; if using flowfile-content as the value for the Destination - property, this value is ignored.
    • - -
    • Supports expression language: false
    • - -
    - -
  • - -
- -

Relationships:

- -
    - -
  • failure -
      - -
    • If the XQuery cannot be evaluated against the content - of the FlowFile, then the FlowFile follows this relationship. For - example, if the FlowFile does not contain valid XML.
    • - -
    - -
  • - -
  • matched -
      - -
    • If the XQuery is successfully evaluated and the - FlowFile is modified as a result, then the FlowFile follows this - relationship.
    • - -
    - -
  • - -
  • unmatched -
      - -
    • If the XQuery does not match the content of the - FlowFile, then the FlowFile follows this relationship.
    • - -
    - -
  • - -
- -

Examples:

- -

This processor produces one attribute or FlowFile per - XQueryResult. If only one attribute or FlowFile is desired, the following - examples demonstrate how this can be achieved using the XQuery - language. The examples below reference the following sample XML:

- -

-  <?xml version="1.0" encoding="UTF-8"?>
-  <?xml-stylesheet type="text/xsl" href="foo.xsl"?>
-  <ns:fruitbasket xmlns:ns="http://namespace/1">
-    <fruit taste="crisp">    
-      <!-- Apples are my favorite-->   
-      <name>apple</name>   
-      <color>red</color>  
-    </fruit>  
-    <fruit>   
-      <name>apple</name>   
-      <color>green</color>  
-    </fruit>  
-    <fruit>   
-      <name>banana</name>   
-      <color>yellow</color>  
-    </fruit>  
-    <fruit taste="sweet">   
-      <name>orange</name>   
-      <color>orange</color>  
-    </fruit>  
-    <fruit>   
-      <name>blueberry</name>   
-      <color>blue</color>  
-    </fruit>  
-      <fruit taste="tart">   
-      <name>raspberry</name>   
-      <color>red</color>  
-    </fruit>  
-    <fruit>   
-      <name>none</name>    
-      <color/>  
-    </fruit>
-  </ns:fruitbasket>
-
- -

-

    -
  • - XQuery to return all "fruit" nodes individually (7 Results): -
    • //fruit
    -
  • -
  • - XQuery to return only the first "fruit" node (1 Result): -
    • //fruit[1]
    -
  • -
  • - XQuery to return only the last "fruit" node (1 Result): -
    • //fruit[count(//fruit)]
    -
  • -
  • - XQuery to return all "fruit" nodes, wrapped in a "basket" tag (1 Result): -
    • <basket>{//fruit}</basket>
    -
  • -
  • - XQuery to return all "fruit" names individually (7 Results): -
    • //fruit/text()
    -
  • -
  • - XQuery to return only the first "fruit" name (1 Result): -
    • //fruit[1]/text()
    -
  • -
  • - XQuery to return only the last "fruit" name (1 Result): -
    • //fruit[count(//fruit)]/text()
    -
  • -
  • - XQuery to return all "fruit" names as a comma separated list (1 Result): -
    • string-join((for $x in //fruit return $x/name/text()), ', ')
    -
  • -
  • - XQuery to return all "fruit" colors and names as a comma separated list (1 Result): -
    • string-join((for $y in (for $x in //fruit return string-join(($x/color/text() , $x/name/text()), ' ')) return $y), ', ')
    -
  • -
  • - XQuery to return all "fruit" colors and names as a comma separated list (1 Result): -
    • string-join((for $y in (for $x in //fruit return string-join(($x/color/text() , $x/name/text()), ' ')) return $y), ', ')
    -
  • -
  • - XQuery to return all "fruit" colors and names as a new line separated list (1 Result): -
    • string-join((for $y in (for $x in //fruit return string-join(($x/color/text() , $x/name/text()), ' ')) return $y), '\n')
    -
  • -
-

- - diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.ExecuteStreamCommand/additionalDetails.html b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.ExecuteStreamCommand/additionalDetails.html new file mode 100644 index 0000000000..0aa703b673 --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.ExecuteStreamCommand/additionalDetails.html @@ -0,0 +1,57 @@ + + + + + + ExecuteStreamCommand + + + + + + + +

+ Creates Attributes: +

+ + + + + + + + + + + + + + + + + + + + + + + + + +
Attribute NameDescription
execution.commandThe name of the command executed to create the new FlowFile
execution.command.argsThe semi-colon delimited list of arguments
execution.statusThe exit status code returned from executing the command
execution.errorAny error messages returned from executing the command
+ + diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.ExecuteStreamCommand/index.html b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.ExecuteStreamCommand/index.html deleted file mode 100644 index 94d0f03fb7..0000000000 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.ExecuteStreamCommand/index.html +++ /dev/null @@ -1,111 +0,0 @@ - - - - - - ExecuteStreamCommand - - - - - - -

Description:

-

This processor executes an external command on the contents of a FlowFile, and creates a new FlowFile with the - results of the command.

-

- Creates Attributes: -

- - - - - - - - - - - - - - - - - - - - - - - - - -
Attribute NameDescription
execution.commandThe name of the command executed to create the new FlowFile
execution.command.argsThe semi-colon delimited list of arguments
execution.statusThe exit status code returned from executing the command
execution.errorAny error messages returned from executing the command
-

- Properties: -

-

In the list below, the names of required properties appear - in bold. Any other properties (not in bold) are considered optional. - If a property has a default value, it is indicated. If a property - supports the use of the NiFi Expression Language (or simply, - "expression language"), that is also indicated.

-
    -
  • Command Path -
      -
    • Specifies the command to be executed; if just the name of an executable is provided, it must be in the user's - environment PATH.
    • -
    • Default value: none
    • -
    • Supports expression language: true
    • -
    -
  • -
  • Command Arguments -
      -
    • The arguments to supply to the executable delimited by the ';' character. Each argument may be an Expression Language - statement.
    • -
    • Default value: none
    • -
    • Supports expression language: true
    • -
    -
  • -
  • Working Directory -
      -
    • The directory to use as the current working directory when executing the command. If the directory does not exist, - then the process's current working directory will be used.
    • -
    • Default value: none (which means whatever NiFi's current working directory is...probably the root of the NiFi - installation directory.)
    • -
    • Supports expression language: true
    • -
    -
  • - -
- -

- Relationships: -

-
    -
  • original -
      -
    • The destination path for the original incoming FlowFile. This FlowFile will have the created attributes listed above.
    • -
    -
  • -
  • output-stream -
      -
    • The destination path for the FlowFile created from the command's output. This FlowFile will have the created attributes listed above.
    • -
    -
  • -
- - diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.GenerateFlowFile/index.html b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.GenerateFlowFile/index.html deleted file mode 100644 index ef1f26b539..0000000000 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.GenerateFlowFile/index.html +++ /dev/null @@ -1,64 +0,0 @@ - - - - - - GenerateFlowFile - - - - - -

Description:

-

- This processor creates FlowFiles of random data to be used for load testing purposes. -

-

- Properties: -

-

- In the list below, the names of required properties appear in bold. Any other properties (not in bold) are considered optional. If a property has a default value, it is indicated. If a property supports the use of the NiFi Expression Language (or simply, "expression language"), that is also indicated. -

-
    -
  • File Size -
      -
    • The file size for each generated FlowFile. The value must be a non-negative integer, followed by a data unit, such as B, KB, MB, GB, TB. Example: 3 KB.
    • -
    • Default value: no default
    • -
    • Supports expression language: false
    • -
  • -
  • Batch Size -
      -
    • The number of FlowFiles to be transferred in each invocation. The value must be a non-negative integer.
    • -
    • Default value: 1
    • -
    • Supports expression language: false
    • -
  • -
- -

- Relationships: -

-
    -
  • success -
      -
    • If FlowFiles are successfully generated, they follow this relationship.
    • -
  • -
- - - - - - diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.GetFTP/additionalDetails.html b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.GetFTP/additionalDetails.html new file mode 100644 index 0000000000..92d69fd1c0 --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.GetFTP/additionalDetails.html @@ -0,0 +1,74 @@ + + + + + + GetFTP + + + + + + +

+ Modifies Attributes: +

+ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
Attribute NameDescription
filenameThe filename is set to the name of the file on the remote server
pathThe path is set to the path of the file's + directory on the remote server. For example, if the <Remote Path> + property is set to /tmp, files picked up from /tmp + will have the path attribute set to /tmp. If the + <Search Recursively> property is set to true + and a file is picked up from /tmp/abc/1/2/3, then the path + attribute will be set to /tmp/abc/1/2/3
file.lastModifiedTimeThe date and time that the source file was last modified.
file.ownerThe numeric owner id of the source file.
file.groupThe numeric group id of the source file.
file.permissionsThe read/write/execute permissions of the source file.
absolute.pathThe full/absolute path from where a file was picked up. The current 'path' attribute is still populated, but may be a relative path.
+ + diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.GetFTP/index.html b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.GetFTP/index.html deleted file mode 100644 index e5d3af9cee..0000000000 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.GetFTP/index.html +++ /dev/null @@ -1,227 +0,0 @@ - - - - - - GetFTP - - - - - -

Description:

-

This processor fetches files from an FTP server and creates - FlowFiles from them.

- -

- Modifies Attributes: -

- - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
Attribute NameDescription
filenameThe filename is set to the name of the file on the remote server
pathThe path is set to the path of the file's - directory on the remote server. For example, if the <Remote Path> - property is set to /tmp, files picked up from /tmp - will have the path attribute set to /tmp. If the - <Search Recursively> property is set to true - and a file is picked up from /tmp/abc/1/2/3, then the path - attribute will be set to /tmp/abc/1/2/3
file.lastModifiedTimeThe date and time that the source file was last modified.
file.ownerThe numeric owner id of the source file.
file.groupThe numeric group id of the source file.
file.permissionsThe read/write/execute permissions of the source file.
absolute.pathThe full/absolute path from where a file was picked up. The current 'path' attribute is still populated, but may be a relative path.
- -

- Properties: -

-

In the list below, the names of required properties appear - in bold. Any other properties (not in bold) are considered optional. - If a property has a default value, it is indicated. If a property - supports the use of the NiFi Expression Language (or simply, - "expression language"), that is also indicated.

-
    -
  • Hostname -
      -
    • The fully qualified hostname or IP address of the remote - system.
    • -
    • Default value: no default
    • -
    • Supports expression language: true
    • -
  • -
  • Port -
      -
    • The port that the remote system is listening on for file transfers.
    • -
    • Default value: 21
    • -
    • Supports expression language: false
    • -
  • -
  • Username -
      -
    • The username for the user account.
    • -
    • Default value: no default
    • -
    • Supports expression language: false
    • -
  • -
  • Password -
      -
    • The password for the user account.
    • -
    • Default value: no default
    • -
    • Supports expression language: false
    • -
    -
  • -
  • Connection Mode -
      -
    • The FTP connection mode. Valid options are: Active or Passive. For most clients, this should be set to Passive.
    • -
    • Default value: Passive
    • -
    • Supports expression language: false
    • -
  • -
  • Transfer Mode -
      -
    • The FTP transfer mode. Valid options are: Binary or ASCII.
    • -
    • Default value: Binary
    • -
    • Supports expression language: false
    • -
  • -
  • Remote Path -
      -
    • The path on the remote system from which to pull files. If - not specified, the user's home directory will be used.
    • -
    • Default value: no default
    • -
    • Supports expression language: true
    • -
    -
  • -
  • File Filter Regex -
      -
    • A Java regular expression for filtering filenames to - determine which files to pick up. If a filter is supplied, only - files whose names match the regular expression are picked up.
    • -
    • Default value: no default
    • -
    • Supports expression language: false
    • -
    -
  • -
  • Polling Interval -
      -
    • Determines how long to wait before fetching the listing for - new files. The unit of measure, such as sec for seconds or min for minutes, must be specified.
    • -
    • Default value: 60 sec
    • -
    • Supports expression language: false
    • -
  • -
  • Search Recursively -
      -
    • A Boolean value (true/false), indicating whether to - traverse subdirectories and pull files from them.
    • -
    • Default value: false
    • -
    • Supports expression language: false
    • -
  • -
  • Ignore Dotted Files -
      -
    • A Boolean value (true/false), indicating whether to ignore - files whose names begin with a dot (.), and, therefore, not pick - them up.
    • -
    • Default value: true
    • -
    • Supports expression language: false
    • -
  • -
  • Delete Original -
      -
    • A Boolean value (true/false), indicating whether to delete - the original copy of the file on the remote system after it has - been transferred.
    • -
    • Default value: true
    • -
    • Supports expression language: false
    • -
  • -
  • Connection Timeout -
      -
    • The amount of time to wait before timing out while creating - a connection. The unit of measure, such as sec for seconds or min for minutes, must be specified.
    • -
    • Default value: 30 sec
    • -
    • Supports expression language: false
    • -
  • -
  • Data Timeout -
      -
    • The amount of time to wait before timing out while - transferring data. The unit of measure, such as sec for seconds or min for minutes, must be specified.
    • -
    • Default value: 30 sec
    • -
    • Supports expression language: false
    • -
  • -
  • Max Selects -
      -
    • The maximum number of files to pull in a single connection. -
    • -
    • Default value: 100
    • -
    • Supports expression language: false
    • -
  • -
  • Remote Poll Batch Size -
      -
    • The given value specifies how many file paths to find in a - given directory on the remote system when doing a file listing. - This value should not need to be modified in general; however, when - polling against a remote system with a significantly large number - of files, this value can be critical. Setting this value too high - can result in very poor performance, while setting it too low can - cause the flow to be slower than usual.
    • -
    • Default value: 5000
    • -
    • Supports expression language: false
    • -
  • -
  • Use Natural Ordering -
      -
    • A Boolean value (true/false), indicating whether to pull - files in the order in which they are naturally listed. If false, - the order in which to pull files is not defined.
    • -
    • Default value: false
    • -
    • Supports expression language: false
    • -
  • -
-

- Relationships: -

-
    -
  • success -
      -
    • If FlowFiles are successfully retrieved by FTP, they follow - this relationship.
    • -
    -
  • -
- - - - diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.GetFile/additionalDetails.html b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.GetFile/additionalDetails.html new file mode 100644 index 0000000000..76046c6485 --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.GetFile/additionalDetails.html @@ -0,0 +1,79 @@ + + + + + + GetFile + + + + + + +

+ Modifies Attributes: +

+ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
Attribute NameDescription
filenameThe filename is set to the name of the file on disk
pathThe path is set to the relative path of the file's directory on disk. For example, if the <Input Directory> + property is set to /tmp, files picked up from /tmp will have the path attribute set to ./. + If the <Recurse Subdirectories> property is set to true and a file is picked up + from /tmp/abc/1/2/3, then the path attribute will be set to abc/1/2/3 +
file.creationTimeThe date and time that the file was created. May not work on all file systems
file.lastModifiedTimeThe date and time that the file was last modified. May not work on all file systems
file.lastAccessTimeThe date and time that the file was last accessed. May not work on all file systems
file.ownerThe owner of the file. May not work on all file systems
file.groupThe group owner of the file. May not work on all file systems
file.permissionsThe read/write/execute permissions of the file. May not work on all file systems
absolute.pathThe full/absolute path from where a file was picked up. The current 'path' attribute is still populated, but may be a relative path.
+ + diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.GetFile/index.html b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.GetFile/index.html deleted file mode 100644 index 9faa12883b..0000000000 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.GetFile/index.html +++ /dev/null @@ -1,186 +0,0 @@ - - - - - - GetFile - - - - - -

Description:

-

This processor obtains FlowFiles from a local directory. NiFi will need at least read permissions on the files it will pull otherwise it will ignore them.

- -

- Modifies Attributes: -

- - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
Attribute NameDescription
filenameThe filename is set to the name of the file on disk
pathThe path is set to the relative path of the file's directory on disk. For example, if the <Input Directory> - property is set to /tmp, files picked up from /tmp will have the path attribute set to ./. - If the <Recurse Subdirectories> property is set to true and a file is picked up - from /tmp/abc/1/2/3, then the path attribute will be set to abc/1/2/3 -
file.creationTimeThe date and time that the file was created. May not work on all file systems
file.lastModifiedTimeThe date and time that the file was last modified. May not work on all file systems
file.lastAccessTimeThe date and time that the file was last accessed. May not work on all file systems
file.ownerThe owner of the file. May not work on all file systems
file.groupThe group owner of the file. May not work on all file systems
file.permissionsThe read/write/execute permissions of the file. May not work on all file systems
absolute.pathThe full/absolute path from where a file was picked up. The current 'path' attribute is still populated, but may be a relative path.
- -

- Properties: -

-

In the list below, the names of required properties appear - in bold. Any other properties (not in bold) are considered optional. - If a property has a default value, it is indicated. If a property - supports the use of the NiFi Expression Language (or simply, - "expression language"), that is also indicated.

-
    -
  • Input Directory -
      -
    • The input directory from which to pull files.
    • -
    • Default value: no default
    • -
    • Supports expression language: true
    • -
  • -
  • File Filter -
      -
    • Only files whose names match the given regular expression - are picked up.
    • -
    • Default value: [^\.].*
    • -
    • Supports expression language: false
    • -
  • -
  • Keep Source File -
      -
    • A Boolean value (true/false), indicating whether to leave a - copy of the file in the input directory after retrieving it; this - causes the file to be picked up continually and is useful for - testing purposes. If not keeping source file NiFi will need - at least write permissions to the directory it is pulling from - otherwise it will ignore the file.
    • -
    • Default value: false
    • -
    • Supports expression language: false
    • -
  • -
  • Recurse Subdirectories -
      -
    • A Boolean value (true/false), indicating whether to pull - files from subdirectories.
    • -
    • Default value: true
    • -
    • Supports expression language: false
    • -
  • -
  • Polling Interval -
      -
    • The amount of time to wait between directory listings. Valid values must be non-negative integers - followed by a time unit, such as nanos, millis, secs, mins, hrs, days. NOTE: If a directory - listing is empty, the processor will yield for the full yield duration and will not poll during that - time. -
    • -
    • Default value: 0 sec
    • -
    • Supports expression language: false
    • -
  • -
  • Ignore Hidden Files -
      -
    • A Boolean value (true/false), indicating whether to ignore - hidden files.
    • -
    • Default value: true
    • -
    • Supports expression language: false
    • -
  • -
  • Minimum File Age -
      -
    • The minimum age that a file must be in order to be pulled; - any file younger than this amount of time (according to the last - modification date) will be ignored.
    • -
    • Default value: 0 sec
    • -
    • Supports expression language: false
    • -
  • -
  • Maximum File Age -
      -
    • The maximum age that a file must be in order to be pulled; - any file older than this amount of time (according to the last - modification date) will be ignored. The value must be a - non-negative integer, followed by a time unit, such as nanos, - millis, secs, mins, hrs, days.
    • -
    • Default value: no default
    • -
    • Supports expression language: false
    • -
    -
  • -
  • Minimum File Size -
      -
    • The minimum size that a file must be in order to be pulled.
    • -
    • Default value: 0 B
    • -
    • Supports expression language: false
    • -
  • -
  • Maximum File Size -
      -
    • The maximum size that a file must be in order to be pulled. - The value must be a non-negative integer followed by a data size - unit, such as B, KB, MB, GB, TB.
    • -
    • Default value: no default
    • -
    • Supports expression language: false
    • -
    -
  • -
-

- Relationships: -

-
    -
  • success -
      -
    • If FlowFiles are successfully selected, they follow this - relationship.
    • -
    -
  • -
- - - diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.GetHTTP/additionalDetails.html b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.GetHTTP/additionalDetails.html new file mode 100644 index 0000000000..94f639bf67 --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.GetHTTP/additionalDetails.html @@ -0,0 +1,45 @@ + + + + + + GetHTTP + + + + + +

Description:

+ +

+ Modifies Attributes: +

+ + + + + + + + + + + + + +
Attribute NameDescription
filenameThe filename is set to the name of the file on the remote server
+ + diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.GetHTTP/index.html b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.GetHTTP/index.html deleted file mode 100644 index 37eb4796e3..0000000000 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.GetHTTP/index.html +++ /dev/null @@ -1,143 +0,0 @@ - - - - - - GetHTTP - - - - - -

Description:

-

This processor fetches files via HTTP and creates FlowFiles - from them. -

- -

- Modifies Attributes: -

- - - - - - - - - - - - - -
Attribute NameDescription
filenameThe filename is set to the name of the file on the remote server
- -

- Properties: -

-

In the list below, the names of required properties appear - in bold. Any other properties (not in bold) are considered optional. - If a property has a default value, it is indicated. If a property - supports the use of the NiFi Expression Language (or simply, - "expression language"), that is also indicated.

-
    -
  • URL -
      -
    • The URL from which to pull files
    • -
    • Default value: no default
    • -
    • Supports expression language: false
    • -
  • -
  • Filename -
      -
    • The filename to assign to the file when pulled.
    • -
    • Default value: no default
    • -
    • Supports expression language: false
    • -
  • -
  • SSL Context Service -
      -
    • The Controller Service to use for obtaining an SSL Context. The SSL Context controller service is a mechanism for providing all the security properties that allow for secure communications between NiFi extensions and other systems. See the User Guide or the Controller Services documentation via the "help" link in the upper-right corner of the GUI for more information about the StandardSSLContextService. The value for this property is the identifier name that is configured in the StandardSSLContextService.
    • -
    • Default value: no default
    • -
    • Supports expression language: false
    • -
    -
  • -
  • Username -
      -
    • The username required to access the URL.
    • -
    • Default value: no default
    • -
    • Supports expression language: false
    • -
    -
  • -
  • Password -
      -
    • The password required to access the URL.
    • -
    • Default value: no default
    • -
    • Supports expression language: false
    • -
    -
  • -
  • Connection Timeout -
      -
    • The amount of time to wait before timing out while creating - a connection.
    • -
    • Default value: 30 sec
    • -
    • Supports expression language: false
    • -
  • -
  • Data Timeout -
      -
    • The amount of time to wait before timing out while - transferring data.
    • -
    • Default value: 30 sec
    • -
    • Supports expression language: false
    • -
  • -
  • User Agent -
      -
    • What to report as the user agent when a connection is made - to the remote server.
    • -
    • Default value: no default
    • -
    • Supports expression language: false
    • -
    -
  • -
  • Accept Content-Type -
      -
    • If specified, requests will only accept the provided - Content-Type.
    • -
    • Default value: no default
    • -
    • Supports expression language: false
    • -
    -
  • -
  • Follow Redirects -
      -
    • A Boolean value (true/false), indicating whether to follow - the redirect provided by the remote server if that server if that - server sends a 3XX HTTP status code.
    • -
    • Default value: false
    • -
    • Supports expression language: false
    • -
    -
  • -
-

- Relationships: -

-
    -
  • success -
      -
    • All incoming FlowFiles follow this relationship.
    • -
    -
  • -
- - - - diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.GetJMSQueue/index.html b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.GetJMSQueue/index.html deleted file mode 100644 index bcbf0dd540..0000000000 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.GetJMSQueue/index.html +++ /dev/null @@ -1,118 +0,0 @@ - - - - - - GetJMSQueue - - - - - -

Description:

-

- This processor pulls messages from a JMS Queue, creating a FlowFile for each JMS message or bundle of messages, as configured. -

-

- Properties: -

-

- In the list below, the names of required properties appear in bold. Any other properties (not in bold) are considered optional. If a property has a default value, it is indicated. If a property supports the use of the NiFi Expression Language (or simply, "expression language"), that is also indicated. -

-
    -
  • JMS Provider -
      -
    • This property specifies the provider used for the JMS server. Available options include: -
        -
      • ActiveMQ
      • -
    • -
    • Default value: ActiveMQ
    • -
    • Supports expression language: false
    • -
  • -
  • URL -
      -
    • The URL of the JMS server.
    • -
    • Default value: no default
    • -
    • Supports expression language: false
    • -
  • -
  • Destination Name -
      -
    • The name of the JMS queue to use.
    • -
    • Default value: no default
    • -
    • Supports expression language: false
    • -
  • -
  • Communications Timeout -
      -
    • The amount of time to wait when attempting to receive a message before giving up and assuming failure.
    • -
    • Default value: 30 sec
    • -
    • Supports expression language: false
    • -
  • -
  • Message Batch Size -
      -
    • The number of messages to pull in a single iteration of the processor.
    • -
    • Default value: 10
    • -
    • Supports expression language: false
    • -
  • -
  • Username -
      -
    • The username used for authentication and authorization.
    • -
    • Default value: no default
    • -
    • Supports expression language: false
    • -
  • -
  • Password -
      -
    • The password used for authentication and authorization.
    • -
    • Default value: no default
    • -
    • Supports expression language: false
    • -
  • -
  • Acknowledgement Mode -
      -
    • The JMS acknowledgement Mode. Two options exist: Client Acknowledge and Auto Acknowledge. Using Auto Acknowledge can cause messages to be lost upon a restart of NiFi but may provide better performance than Client Acknowledge.
    • -
    • Default value: Client Acknowledge
    • -
    • Supports expression language: false
    • -
  • -
  • Message Selector -
      -
    • The JMS Message Selector to use in order to narrow the messages that are pulled.
    • -
    • Default value: no default
    • -
    • Supports expression language: false
    • -
  • -
  • Copy of JMS Properties Attributes -
      -
    • A Boolean value (true/false), indicating whether the JMS Message Properties should be copied to the FlowFile attributes; if so, each attribute name will be jms.XXX, where XXX is the JMS Property name.
    • -
    • Default value: true
    • -
    • Supports expression language: false
    • -
  • -
  • Client ID Prefix -
      -
    • A human-readable ID that can be used to associate connections with yourself so that the maintainers of the JMS server know who to contact if problems arise.
    • -
    • Default value: no default
    • -
    • Supports expression language: false
    • -
  • -
-

- Relationships: -

-
    -
  • success -
      -
    • All incoming FlowFiles follow this relationship.
    • -
  • -
- - - - diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.GetJMSTopic/index.html b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.GetJMSTopic/index.html deleted file mode 100644 index 3496df8273..0000000000 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.GetJMSTopic/index.html +++ /dev/null @@ -1,122 +0,0 @@ - - - - - - GetJMSTopic - - - - - -

Description:

-

- This processor pulls messages from a JMS Topic, creating a FlowFile for each JMS message or bundle of messages, as configured. -

-

- Properties: -

-

- In the list below, the names of required properties appear in bold. Any other properties (not in bold) are considered optional. If a property has a default value, it is indicated. If a property supports the use of the NiFi Expression Language (or simply, "expression language"), that is also indicated. -

-
    -
  • JMS Provider -
      -
    • This property specifies the provider used for the JMS server. Available options include: -
        -
      • ActiveMQ
      • -
    • -
    • Supports expression language: false
    • -
  • -
  • URL -
      -
    • The URL of the JMS server.
    • -
    • Default value: no default
    • -
    • Supports expression language: false
    • -
  • -
  • Destination Name -
      -
    • The name of the JMS topic to use.
    • -
    • Default value: no default
    • -
    • Supports expression language: false
    • -
  • -
  • Communications Timeout -
      -
    • The amount of time to wait when attempting to receive a message before giving up and assuming failure.
    • -
    • Default value: 30 sec
    • -
    • Supports expression language: false
    • -
  • -
  • Message Batch Size -
      -
    • The number of messages to pull in a single iteration of the processor.
    • -
    • Default value: 10
    • -
    • Supports expression language: false
    • -
  • -
  • Username -
      -
    • The username used for authentication and authorization.
    • -
    • Default value: no default
    • -
    • Supports expression language: false
    • -
  • -
  • Password -
      -
    • The password used for authentication and authorization.
    • -
    • Default value: no default
    • -
    • Supports expression language: false
    • -
  • -
  • Acknowledgement Mode -
      -
    • The JMS acknowledgement Mode. Two options exist: Client Acknowledge and Auto Acknowledge. Using Auto Acknowledge can cause messages to be lost upon a restart of NiFi but may provide better performance than Client Acknowledge.
    • -
    • Default value: Client Acknowledge
    • -
    • Supports expression language: false
    • -
  • -
  • Message Selector -
      -
    • The JMS Message Selector to use in order to narrow the messages that are pulled.
    • -
    • Default value: no default
    • -
    • Supports expression language: false
    • -
  • -
  • Copy of JMS Properties Attributes -
      -
    • A Boolean value (true/false), indicating whether the JMS Message Properties should be copied to the FlowFile attributes; if so, each attribute name will be jms.XXX, where XXX is the JMS Property name.
    • -
    • Default value: true
    • -
    • Supports expression language: false
    • -
  • -
  • Client ID Prefix -
      -
    • A human-readable ID that can be used to associate connections with yourself so that the maintainers of the JMS server know who to contact if problems arise.
    • -
    • Default value: no default
    • -
    • Supports expression language: false
    • -
  • -
  • Use Durable Subscription -
      -
    • A Boolean value (true/false), indicating whether connections to the specified topic will use Durable Subscription so that messages are queued when NiFi is not pulling them.
    • -
    • Default value: false
    • -
    • Supports expression language: false
    • -
  • -
-

- Relationships: -

-
    -
  • success -
      -
    • All incoming FlowFiles follow this relationship.
    • -
  • -
- - - diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.GetSFTP/additionalDetails.html b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.GetSFTP/additionalDetails.html new file mode 100644 index 0000000000..8cb65261c9 --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.GetSFTP/additionalDetails.html @@ -0,0 +1,80 @@ + + + + + + GetSFTP + + + + + + + +

+ Modifies Attributes: +

+ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
Attribute NameDescription
filenameThe filename is set to the name of the file on the remote server
pathThe path is set to the path of the file's + directory on the remote server. For example, if the <Remote Path> + property is set to /tmp, files picked up from /tmp + will have the path attribute set to /tmp. If the + <Search Recursively> property is set to true + and a file is picked up from /tmp/abc/1/2/3, then the path + attribute will be set to /tmp/abc/1/2/3
file.lastModifiedTimeThe date and time that the source file was last modified.
file.ownerThe numeric owner id of the source file.
file.groupThe numeric group id of the source file.
file.permissionsThe read/write/execute permissions of the source file.
absolute.pathThe full/absolute path from where a file was picked up. The current 'path' attribute is still populated, but may be a relative path.
+

+ See Also: +

+ + diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.GetSFTP/index.html b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.GetSFTP/index.html deleted file mode 100644 index 708f5a35bc..0000000000 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.GetSFTP/index.html +++ /dev/null @@ -1,250 +0,0 @@ - - - - - - GetSFTP - - - - - - -

Description:

-

This processor pulls files from an SFTP server and creates - FlowFiles to encapsulate them. -

- -

- Modifies Attributes: -

- - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
Attribute NameDescription
filenameThe filename is set to the name of the file on the remote server
pathThe path is set to the path of the file's - directory on the remote server. For example, if the <Remote Path> - property is set to /tmp, files picked up from /tmp - will have the path attribute set to /tmp. If the - <Search Recursively> property is set to true - and a file is picked up from /tmp/abc/1/2/3, then the path - attribute will be set to /tmp/abc/1/2/3
file.lastModifiedTimeThe date and time that the source file was last modified.
file.ownerThe numeric owner id of the source file.
file.groupThe numeric group id of the source file.
file.permissionsThe read/write/execute permissions of the source file.
absolute.pathThe full/absolute path from where a file was picked up. The current 'path' attribute is still populated, but may be a relative path.
- -

- Properties: -

-

In the list below, the names of required properties appear - in bold. Any other properties (not in bold) are considered optional. - If a property has a default value, it is indicated. If a property - supports the use of the NiFi Expression Language (or simply, - "expression language"), that is also indicated.

-
    -
  • Hostname -
      -
    • The fully qualified hostname or IP address of the remote - system.
    • -
    • Default value: no default
    • -
    • Supports expression language: true
    • -
  • -
  • Port -
      -
    • The port that the remote system is listening on for file transfers.
    • -
    • Default value: 22
    • -
    • Supports expression language: false
    • -
  • -
  • Username -
      -
    • The username for the user account on the remote system.
    • -
    • Default value: no default
    • -
    • Supports expression language: false
    • -
  • -
  • Password -
      -
    • The password for the user account.
    • -
    • Default value: no default
    • -
    • Supports expression language: false
    • -
    -
  • -
  • Private Key Path -
      -
    • The fully qualified path for the private key file.
    • -
    • Default value: no default
    • -
    • Supports expression language: false
    • -
    -
  • -
  • Private Key Passphrase -
      -
    • The password for the private key.
    • -
    • Default value: no default
    • -
    • Supports expression language: false
    • -
    -
  • -
  • Remote Path -
      -
    • The path on the remote system from which to pull files. If - not specified, the user's home directory will be used.
    • -
    • Default value: no default
    • -
    • Supports expression language: true
    • -
    -
  • -
  • File Filter Regex -
      -
    • Provides a Java Regular Expression for filtering Filenames; if a filter is supplied, only files - whose names match that Regular Expression will be fetched
    • -
    • Default value: no default
    • -
    • Supports expression language: false
    • -
    -
  • -
  • Polling Interval -
      -
    • Determines how long to wait between fetching the listing for new files
    • -
    • Default value: 60 secs
    • -
    • Supports expression language: false
    • -
    -
  • -
  • Search Recursively -
      -
    • If true, will pull files from arbitrarily nested subdirectories; otherwise, will not traverse subdirectories
    • -
    • Default value: false
    • -
    • Supports expression language: false
    • -
    -
  • -
  • Ignore Dotted Files -
      -
    • If true, files whose names begin with a dot (\".\") will be ignored
    • -
    • Default value: true
    • -
    • Supports expression language: false
    • -
    -
  • -
  • Delete Original -
      -
    • Determines whether or not the file is deleted from the remote system after it has been successfully transferred
    • -
    • Default value: true
    • -
    • Supports expression language: false
    • -
    -
  • -
  • Connection Timeout -
      -
    • The amount of time to wait before timing out while creating a connection.
    • -
    • Default value: 30 sec
    • -
    • Supports expression language: false
    • -
  • -
  • Data Timeout -
      -
    • The amount of time to wait before timing out while - transferring data.
    • -
    • Default value: 30 sec
    • -
    • Supports expression language: false
    • -
  • -
  • Host Key File -
      -
    • The local file path to the host key file; if not supplied, - no host key file will be used.
    • -
    • Default value: no default
    • -
    • Supports expression language: false
    • -
    -
  • -
  • Max Selects -
      -
    • The maximum number of files to pull in a single connection.
    • -
    • Default value: 100
    • -
    • Supports expression language: false
    • -
  • -
  • Remote Poll Batch Size -
      -
    • The value specifies how many file paths to find in a given directory on the remote system when - doing a file listing. This value in general should not need to be modified but when polling against a - remote system with a tremendous number of files this value can be critical. Setting this value too high - can result very poor performance and setting it too low can cause the flow to be slower than normal.
    • -
    • Default value: 5000
    • -
    • Supports expression language: false
    • -
  • -
  • Strict Host Key Checking -
      -
    • A Boolean value (true/false), indicating whether to apply - strict enforcement of host keys.
    • -
    • Default value: false
    • -
    • Supports expression language: false
    • -
  • -
  • Use Compression -
      -
    • A Boolean value (true/false), indicating whether to use - ZLIB compression when transferring files.
    • -
    • Default value: false
    • -
    • Supports expression language: false
    • -
  • -
  • Use Natural Ordering -
      -
    • If true, will pull files in the order in which they are naturally listed; otherwise, the order in - which the files will be pulled is not defined
    • -
    • Default value: false
    • -
    • Supports expression language: false
    • -
  • -
- -

- Relationships: -

-
    -
  • success -
      -
    • If FlowFiles are successfully transferred, then they follow - this relationship.
    • -
    -
  • -
- -

- See Also: -

-

- - diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.HashAttribute/additionalDetails.html b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.HashAttribute/additionalDetails.html new file mode 100644 index 0000000000..165568c5a4 --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.HashAttribute/additionalDetails.html @@ -0,0 +1,35 @@ + + + + + + HashAttribute + + + + + + + +

+ Modifies Attributes: +

+

+ This Processor adds an attribute whose value is the result of Hashing the existing FlowFile attributes. + The name of this attribute is specified by the <Hash Value Attribute Key> property. +

+ + diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.HashAttribute/index.html b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.HashAttribute/index.html deleted file mode 100644 index a2d4fd2b52..0000000000 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.HashAttribute/index.html +++ /dev/null @@ -1,88 +0,0 @@ - - - - - - HashAttribute - - - - - -

Description:

-

This processor hashes together the key/value pairs of - several FlowFile attributes and adds the hash as a new attribute. The - user may add optional properties such that the name of each property - is the name of a FlowFile attribute to consider and the value of the - property is a regular expression that, if matched by the attribute - value, causes that attribute to be used as part of the hash. If the - regular expression contains a capturing group, only the value of the - capturing group is used. -

- -

- Modifies Attributes: -

-

- This Processor adds an attribute whose value is the result of Hashing the existing FlowFile attributes. - The name of this attribute is specified by the <Hash Value Attribute Key> property. -

- -

- Properties: -

-

In the list below, the names of required properties appear - in bold. Any other properties (not in bold) are considered optional. - If a property has a default value, it is indicated. If a property - supports the use of the NiFi Expression Language (or simply, - "expression language"), that is also indicated.

-
    -
  • Hash Value Attribute Key -
      -
    • The name of the FlowFile attribute whose hash value should - be stored.
    • -
    • Default value: no default
    • -
    • Supports expression language: false
    • -
  • -
  • User-Added Properties -
      -
    • See description above.
    • -
    -
  • -
- -

- Relationships: -

-
    -
  • success -
      -
    • If the processor successfully hashes the key/value pairs of - the FlowFile, then the FlowFile follows this relationship.
    • -
    -
  • -
  • failure -
      -
    • If something prevents the processor from successfully - hashing the key/value pairs of the FlowFile, then the FlowFile - follows this relationship.
    • -
    -
  • -
- - - - diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.HashContent/additionalDetails.html b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.HashContent/additionalDetails.html new file mode 100644 index 0000000000..8a3d1cbd7f --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.HashContent/additionalDetails.html @@ -0,0 +1,35 @@ + + + + + + HashContent + + + + + + + +

+ Modifies Attributes: +

+

+ This Processor adds an attribute whose value is the result of Hashing the existing FlowFile attributes. + The name of this attribute is specified by the <Hash Attribute Name> property. +

+ + diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.HashContent/index.html b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.HashContent/index.html deleted file mode 100644 index a14265f092..0000000000 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.HashContent/index.html +++ /dev/null @@ -1,89 +0,0 @@ - - - - - - HashContent - - - - - -

Description:

-

- This processor calculates a hash value for the content of a - FlowFile and puts the hash value on the FlowFile as an attribute whose - name is determined by the Hash Attribute Name - property. -

- -

- Modifies Attributes: -

-

- This Processor adds an attribute whose value is the result of Hashing the existing FlowFile attributes. - The name of this attribute is specified by the <Hash Attribute Name> property. -

- - -

- Properties: -

-

In the list below, the names of required properties appear - in bold. Any other properties (not in bold) are considered optional. - If a property has a default value, it is indicated. If a property - supports the use of the NiFi Expression Language (or simply, - "expression language"), that is also indicated.

-
    -
  • Hash Attribute Name -
      -
    • The name of the FlowFile attribute to which the hash value - should be written. If the value already exists, it will be - overwritten.
    • -
    • Default value: hash.value
    • -
    • Supports expression language: false
    • -
  • -
  • Hash Algorithm -
      -
    • The hashing algorithm that should be used to perform the - hashing function.
    • -
    • Default value: MD5
    • -
    • Supports expression language: false
    • -
  • -
- - -

- Relationships: -

-
    -
  • success -
      -
    • If the FlowFile is processed successfully, then the - FlowFile follows this relationship.
    • -
    -
  • -
  • failure -
      -
    • If the FlowFile is not processed successfully, then the - FlowFile follows this relationship.
    • -
    -
  • -
- - - - diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.IdentifyMimeType/index.html b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.IdentifyMimeType/additionalDetails.html similarity index 73% rename from nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.IdentifyMimeType/index.html rename to nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.IdentifyMimeType/additionalDetails.html index 120b69cd84..1fb55667f7 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.IdentifyMimeType/index.html +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.IdentifyMimeType/additionalDetails.html @@ -42,6 +42,7 @@

The following MIME Types are detected: +

  • application/gzip
  • application/bzip2
  • @@ -73,7 +74,7 @@
  • application/zip
  • application/x-lzh
-

+

Modifies Attributes: @@ -93,44 +94,5 @@ - -

- Properties: -

-

In the list below, the names of required properties appear - in bold. Any other properties (not in bold) are considered optional. - If a property has a default value, it is indicated. If a property - supports the use of the NiFi Expression Language (or simply, - "expression language"), that is also indicated.

-
    -
  • Identify ZIP -
      -
    • A Boolean value (true/false), indicating whether to attempt in depth identification - of ZIP MIME types.
    • -
    • Default value: false
    • -
    • Supports expression language: false
    • -
  • -
  • Identify TAR -
      -
    • A Boolean value (true/false), indicating whether to attempt in depth identification - of TAR MIME types.
    • -
    • Default value: false
    • -
    • Supports expression language: false
    • -
  • -
- - -

- Relationships: -

-
    -
  • success -
      -
    • All FlowFiles follow this relationship, regardless of - whether the MIME type was detected.
    • -
    -
  • -
- diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.InvokeHTTP/additionalDetails.html b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.InvokeHTTP/additionalDetails.html new file mode 100644 index 0000000000..f7f06a047b --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.InvokeHTTP/additionalDetails.html @@ -0,0 +1,64 @@ + + + + + + InvokeHTTP - a swiss army http client processor + + + + + +

+ Adds Attributes: +

+ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
Attribute NameDescription
invokehttp.status.codeThe status code that is returned.
invokehttp.status.messageThe status message that is returned.
invokehttp.response.bodyThe response body.
invokehttp.request.urlThe request URL.
invokehttp.tx.idThe transaction ID that is returned after reading the response.
invokehttp.remote.dnThe DN of the remote server.
+ + diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.InvokeHTTP/index.html b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.InvokeHTTP/index.html deleted file mode 100644 index 2907034bfa..0000000000 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.InvokeHTTP/index.html +++ /dev/null @@ -1,181 +0,0 @@ - - - - - - InvokeHTTP - a swiss army http client processor - - - - -

Description:

-

- Making requests to remote HTTP servers. Supporting common HTTP methods. - Storing results as new flowfiles upon success. Routing to failure on error. -

-

- An HTTP client processor that converts FlowFile attributes to HTTP headers with configurable HTTP method, URL, etc. -

- -

- Adds Attributes: -

- - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
Attribute NameDescription
invokehttp.status.codeThe status code that is returned.
invokehttp.status.messageThe status message that is returned.
invokehttp.response.bodyThe response body.
invokehttp.request.urlThe request URL.
invokehttp.tx.idThe transaction ID that is returned after reading the response.
invokehttp.remote.dnThe DN of the remote server.
- -

- Properties: -

-
    -
  • - HTTP Method -
      -
    • The HTTP request method (e.g., GET, POST, PUT, DELETE, HEAD, OPTIONS)
    • -
    • Default value: GET
    • -
    • Supports expression language: false
    • -
    -
  • -
  • - Remote URL -
      -
    • The remote URL that will be conneted to, including scheme, host, port, and path.
    • -
    • Default value: no default
    • -
    • Supports expression language: true
    • -
    -
  • -
  • - SSL Context Service -
      -
    • The Controller Service to use for obtaining an SSL Context. The SSL Context controller service is a mechanism for providing all the security properties - that allow for secure communications between NiFi extensions and other systems. See the User Guide or the Controller Services documentation (via the "help" link in the - upper-right corner of the GUI for more information about the StandardSSLContextService. The value for this property is the identifier name that is configured in - the StandardSSLContextService.
    • -
    • Default value: no default
    • -
    • Supports expression language: false
    • -
    -
  • -
  • - Connection Timeout -
      -
    • The amount of time to wait before timing out while creating a connection.
    • -
    • Default value: 5 secs
    • -
    • Supports expression language: false
    • -
    -
  • -
  • - Read Timeout -
      -
    • The amount of time to wait before timing out while waiting for a response from the remote service.
    • -
    • Default value: 15 secs
    • -
    • Supports expression language: false
    • -
    -
  • -
  • - Include Data Header -
      -
    • A Boolean value (true/false), indicating whether to include an RFC-2616 date header in the request.
    • -
    • Default value: True
    • -
    • Supports expression language: false
    • -
    -
  • -
  • - Follow Redirects -
      -
    • A Boolean value (true/false), indicating whether to follow HTTP redirects issued by the remote server.
    • -
    • Default value: True
    • -
    • Supports expression language: false
    • -
    -
  • -
  • - Attributes to Send -
      -
    • A regular expression that defines which attributes to send as HTTP headers in the request. If not defined, no attributes are sent as headers.
    • -
    • Default value: no default
    • -
    • Supports expression language: false
    • -
    -
  • -
-

- Relationships: -

-
    -
  • - Original -
      -
    • Original FlowFile will be routed upon success (2xx status codes).
    • -
    -
  • -
  • - Response -
      -
    • Response FlowFile will be routed upon success (2xx status codes).
    • -
    -
  • -
  • - Retry -
      -
    • FlowFile will be routed on any status code that can be retried (5xx status codes).
    • -
    -
  • -
  • - No Retry -
      -
    • FlowFile will be routed on any status code that should NOT be retried (1xx, 3xx, 4xx status codes).
    • -
    -
  • -
  • - Failure -
      -
    • FlowFile will be routed on any type of connection failure, timeout or general exception.
    • -
    -
  • -
- - - diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.ListenHTTP/index.html b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.ListenHTTP/index.html deleted file mode 100644 index 49e85cc825..0000000000 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.ListenHTTP/index.html +++ /dev/null @@ -1,86 +0,0 @@ - - - - - - ListenHTTP - - - - - -

Description:

-

- This processor starts an HTTP service that is used to receive FlowFiles from remote sources. The URL of the service is http://{hostname}:{port}/contentListener. -

-

- Properties: -

-

- In the list below, the names of required properties appear in bold. Any other properties (not in bold) are considered optional. If a property has a default value, it is indicated. If a property supports the use of the NiFi Expression Language (or simply, "expression language"), that is also indicated. -

-
    -
  • Listening Port -
      -
    • The port to listen on for incoming connections.
    • -
    • Default value: no default
    • -
    • Supports expression language: false
    • -
  • -
  • Max Data to Receive per Second -
      -
    • The maximum amount of data to receive per second; this allows the bandwidth to be throttled to a specified data rate. If not specified, the data rate is not throttled.
    • -
    • Default value: no default
    • -
    • Supports expression language: false
    • -
  • -
  • SSL Context Service -
      -
    • The Controller Service to use for obtaining an SSL Context. The SSL Context controller service is a mechanism for providing all the security properties that allow for secure communications between NiFi extensions and other systems. See the User Guide or the Controller Services documentation via the "help" link in the upper-right corner of the GUI for more information about the StandardSSLContextService. The value for this property is the identifier name that is configured in the StandardSSLContextService.
    • -
    • Default value: no default
    • -
    • Supports expression language: false
    • -
  • -
  • Authorized DN Pattern -
      -
    • A regular expression to apply against the distinguished anme (DN) of incoming connections. If the pattern does not match the DN, the connection is refused.
    • -
    • Default value: .*
    • -
    • Supports expression language: false
    • -
  • -
  • HTTP Headers to receive as Attributes (Regex) -
      -
    • Specifies the Regular Expression that determines the names of HTTP Headers that should be passed along as FlowFile attributes
    • -
    • Default value: no default
    • -
    • Supports expression language: false
    • -
  • -
  • Max Unconfirmed FlowFile Time -
      -
    • The maximum amount of time to wait for a FlowFile to be confirmed before it is removed from the cache.
    • -
    • Default value: 60 secs
    • -
    • Supports expression language: false
    • -
  • -
- -

- Relationships: -

-
    -
  • success -
      -
    • If the processor successfully receives files via an HTTP service, then the FlowFiles follow this relationship.
    • -
  • -
- - - - diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.ListenUDP/index.html b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.ListenUDP/index.html deleted file mode 100644 index c50e79d651..0000000000 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.ListenUDP/index.html +++ /dev/null @@ -1,144 +0,0 @@ - - - - - - ListenUDP - - - - - - -

Description:

-

- This processor listens for Datagram Packets on a given port and concatenates the contents of - those packets together generating FlowFiles roughly as often as the internal buffer fills up or until - no more data is currently available. -

-

- Properties: -

-

- In the list below, the names of required properties appear in bold. Any other properties (not in bold) are considered optional. If a property has a default value, it is indicated. If a property supports the use of the NiFi Expression Language (or simply, "expression language"), that is also indicated. -

-
    -
  • - Port -
      -
    • The port to listen on for data packets. Must be known by senders of Datagrams. May be a system - property or environment variable.
    • -
    • Default value: no default
    • -
    • Supports expression language: false
    • -
    -
  • -
  • - Receive Timeout -
      -
    • The time out period when waiting to receive data from the socket. Specify units.
    • -
    • Default value: 5 secs
    • -
    • Supports expression language: false
    • -
    -
  • -
  • - Max Buffer Size -
      -
    • Determines the size each receive buffer may be. Specify units.
    • -
    • Default value: 1 MB
    • -
    • Supports expression language: false
    • -
    -
  • -
  • - FlowFile Size Trigger -
      -
    • Determines the (almost) upper bound size at which a flow file would be generated. A flow file will get made even if - this value isn't reached if there is no more data streaming in and this value may be exceeded by the size of a - single packet. Specify units.
    • -
    • Default value: 1 MB
    • -
    • Supports expression language: false
    • -
    -
  • -
  • - Max size of UDP Buffer -
      -
    • The maximum UDP buffer size that should be used. This is a suggestion to the Operating System - to indicate how big the udp socket buffer should be. Specify units.
    • -
    • Default value: 1 MB
    • -
    • Supports expression language: false
    • -
    -
  • -
  • - Receive Buffer Count -
      -
    • Number of receiving buffers to be used to accept data from the socket. Higher numbers means more - ram is allocated but can allow better throughput.
    • -
    • Default value: 4
    • -
    • Supports expression language: false
    • -
    -
  • -
  • - Channel Reader Interval -
      -
    • Scheduling interval for each read channel. Specify units.
    • -
    • Default value: 50 millisecs
    • -
    • Supports expression language: false
    • -
    -
  • -
  • - FlowFiles Per Session -
      -
    • The number of flow files per session. Higher number is more efficient, but will lose more data - if a problem occurs that causes a rollback of a session.
    • -
    • Default value: 10
    • -
    • Supports expression language: false
    • -
    -
  • -
  • - Sending Host -
      -
    • IP, or name, of a remote host. Only Datagrams from the specified Sending Host Port and this host - will be accepted. Improves Performance. May be a system property or an environment variable. If - this is specified, the Sending Host Port must be specified as well.
    • -
    • Default value: none
    • -
    • Supports expression language: true
    • -
    -
  • -
  • - Sending Host Port -
      -
    • Port being used by remote host to send Datagrams. Only Datagrams from the specified Sending Host - and this port will be accepted. Improves Performance. May be a system property or an environment - variable.If this is specified, the Sending Host must be specified as well.
    • -
    • Default value: none
    • -
    • Supports expression language: true
    • -
    -
  • -
- -

- Relationships: -

-
    -
  • - success -
      -
    • Used when file is successfully created and filled with UDP packets.
    • -
    -
  • -
- - - diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.LogAttribute/index.html b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.LogAttribute/index.html deleted file mode 100644 index 33ab03cad3..0000000000 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.LogAttribute/index.html +++ /dev/null @@ -1,80 +0,0 @@ - - - - - - LogAttribute - - - - - - -

Description:

-

- This processor reads the attributes on incoming FlowFiles and prints those attributes and their values to the log at the logging level specified by the user.

-

- Properties: -

-

- In the list below, the names of required properties appear in bold. Any other properties (not in bold) are considered optional. If a property has a default value, it is indicated. If a property supports the use of the NiFi Expression Language (or simply, "expression language"), that is also indicated. -

-
    -
  • Attributes to Ignore -
      -
    • A comma-separated list of attributes to ignore. If not specified, no attributes will be ignored.
    • -
    • Default value: no default
    • -
    • Supports expression language: false
    • -
  • -
  • Attributes to Log -
      -
    • A comma-separated list of attributes to log. If not specified, all attributes are logged.
    • -
    • Default value: no default
    • -
    • Supports expression language: false
    • -
  • -
  • Log Payload -
      -
    • A Boolean value (true/false), indicating whether to log the payload of incoming FlowFiles.
    • -
    • Default value: false
    • -
    • Supports expression language: false
    • -
  • -
  • Log Level -
      -
    • The log level at which to log the attributes. Acceptable values are: -
        -
      • trace
      • -
      • debug
      • -
      • info
      • -
    • -
    • Default value: info
    • -
    • Supports expression language: false
    • -
  • -
- -

- Relationships: -

-
    -
  • success -
      -
    • All outgoing FlowFiles follow this relationship.
    • -
  • -
- - - - - diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.MergeContent/additionalDetails.html b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.MergeContent/additionalDetails.html new file mode 100644 index 0000000000..68f70076be --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.MergeContent/additionalDetails.html @@ -0,0 +1,115 @@ + + + + + + MergeContent + + + + + + + +

+ Uses Attributes: +

+ + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
Attribute NameDescription
fragment.identifierApplicable only if the <Merge Strategy> property is set to Defragment. + All FlowFiles with the same value for this attribute will be bundled together. +
fragment.indexApplicable only if the <Merge Strategy> property is set to Defragment. + This attribute must be present on all FlowFiles with the same value for the fragment.identifier + attribute and must be a unique integer between 0 and the value of the fragment.count attribute. + This attribute indicates the order in which the fragments should be assembled. +
fragment.countApplicable only if the <Merge Strategy> property is set to Defragment. + This attribute must be present on all FlowFiles with the same value for the fragment.identifier + attribute. All FlowFiles in the same bundle must have the same value for this attribute. The value of + this attribute indicates how many FlowFiles should be expected in the given bundle. +
segment.original.filenameApplicable only if the <Merge Strategy> property is set to Defragment. + This attribute must be present on all FlowFiles with the same value for the fragment.identifier + attribute. All FlowFiles in the same bundle must have the same value for this attribute. The value of + this attribute will be used for the filename of the completed merged FlowFile. +
tar.permissionsApplicable only if the <Merge Format> property is set to TAR. + The value of this attribute must be 3 characters; each character must be in the range 0 to 7 (inclusive) + and indicates the file permissions that should be used for the FlowFile's TAR entry. If this attribute + is missing or has an invalid value, the default value of 644 will be used. +
+ +

+ Modifies Attributes: +

+ + + + + + + + + + + + + + + + + + + + + +
Attribute NameDescription
filenameWhen more than 1 file is merged, the filename comes from the segment.original.filename attribute. + If that attribute does not exist in the source FlowFiles, then the filename is set to the number of + nanoseconds matching system time. Then a filename extension may be applied: +
    +
  • if Merge Format is TAR, then the filename will be appended with .tar
  • +
  • if Merge Format is ZIP, then the filename will be appended with .zip
  • +
  • if Merge Format is FlowFileStream, then the filename will be appended with .pkg
  • +
+
merge.countThe number of FlowFiles that were merged into this bundle.
merge.bin.ageThe age of the bin, in milliseconds, when it was merged and output. Effectively this is the + greatest amount of time that any FlowFile in this bundle remained waiting in this processor before it + was output.
+ + diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.MergeContent/index.html b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.MergeContent/index.html deleted file mode 100644 index 3e02b816f8..0000000000 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.MergeContent/index.html +++ /dev/null @@ -1,347 +0,0 @@ - - - - - - MergeContent - - - - - - -

Description:

-

This processor merges a group of FlowFiles together into a - "Bundle" based on a user-defined strategy and packages them into a - single FlowFile. It is recommended that the processor be configured - with only a single incoming connection, as groups of FlowFiles will - not be created from FlowFiles in different connections. This processor - updates the mime.type attribute as appropriate. After files have been merged by this processor, they can be - unpackaged later using the UnpackContent processor.

- -

- Uses Attributes: -

- - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
Attribute NameDescription
fragment.identifierApplicable only if the <Merge Strategy> property is set to Defragment. - All FlowFiles with the same value for this attribute will be bundled together. -
fragment.indexApplicable only if the <Merge Strategy> property is set to Defragment. - This attribute must be present on all FlowFiles with the same value for the fragment.identifier - attribute and must be a unique integer between 0 and the value of the fragment.count attribute. - This attribute indicates the order in which the fragments should be assembled. -
fragment.countApplicable only if the <Merge Strategy> property is set to Defragment. - This attribute must be present on all FlowFiles with the same value for the fragment.identifier - attribute. All FlowFiles in the same bundle must have the same value for this attribute. The value of - this attribute indicates how many FlowFiles should be expected in the given bundle. -
segment.original.filenameApplicable only if the <Merge Strategy> property is set to Defragment. - This attribute must be present on all FlowFiles with the same value for the fragment.identifier - attribute. All FlowFiles in the same bundle must have the same value for this attribute. The value of - this attribute will be used for the filename of the completed merged FlowFile. -
tar.permissionsApplicable only if the <Merge Format> property is set to TAR. - The value of this attribute must be 3 characters; each character must be in the range 0 to 7 (inclusive) - and indicates the file permissions that should be used for the FlowFile's TAR entry. If this attribute - is missing or has an invalid value, the default value of 644 will be used. -
- -

- Modifies Attributes: -

- - - - - - - - - - - - - - - - - - - - - -
Attribute NameDescription
filenameWhen more than 1 file is merged, the filename comes from the segment.original.filename attribute. - If that attribute does not exist in the source FlowFiles, then the filename is set to the number of - nanoseconds matching system time. Then a filename extension may be applied: -
    -
  • if Merge Format is TAR, then the filename will be appended with .tar
  • -
  • if Merge Format is ZIP, then the filename will be appended with .zip
  • -
  • if Merge Format is FlowFileStream, then the filename will be appended with .pkg
  • -
-
merge.countThe number of FlowFiles that were merged into this bundle.
merge.bin.ageThe age of the bin, in milliseconds, when it was merged and output. Effectively this is the - greatest amount of time that any FlowFile in this bundle remained waiting in this processor before it - was output.
- -

- Properties: -

-

In the list below, the names of required properties appear - in bold. Any other properties (not in bold) are considered optional. - If a property has a default value, it is indicated. If a property - supports the use of the NiFi Expression Language (or simply, - "expression language"), that is also indicated.

-
    -
  • Merge Strategy -
      -
    • Specifies the algorithm used to merge content. The - following Merge Strategies are available: -
        -
      • Bin-Packing Algorithm - This strategy uses the - <Minimum Number of Entries> and <Maximum Number of - Entries> properties to determine how many FlowFiles should be - used to create a Bundle.
      • -
      • Defragment - This strategy is used when - recombining a set of FlowFiles that have been broken apart - previously. When using this value, the <Minimum Number of - Entries> and <Maximum Number of Entries> properties are - ignored. In lieu of using these properties, the pocessor - determines the number of FlowFiles to use for a bundle by - examining the fragment.count attribute of the - FlowFiles.
      • -
      -
    • -
    • Default value: Bin-Packing Algorithm
    • -
    • Supports expression language: false
    • -
  • -
  • Merge Format -
      -
    • Determines the format that will be used to merge the - content. Options include the following values: -
        -
      • Binary Concatenation - FlowFiles will be combined - by concatenating their content. The ordering of the FlowFiles is - non-deterministic.
      • - -
      • TAR - FlowFiles will be combined by creating a TAR - file. If the <Keep Path> property is set to true, - the FlowFiles' paths will determine the folder in the TAR file - where the FlowFile's content will be stored. Otherwise, all - FlowFiles' content will be stored in the TAR file's root - directory. If a FlowFile that is to be bundled has an attribute - with the name tar.permissions and that attribute's - value is a 3-digit number (with each number in the range of 0-7), - the value will be applied as the File Permissions for that TAR - entry.
      • - -
      • ZIP - FlowFiles will be combined by creating a ZIP - file. If the <Keep Path> property is set to true, - the FlowFiles' paths will determine the folder in the ZIP file - where the FlowFile's content will be stored. Otherwise, all - FlowFiles' content will be stored in the ZIP file's root - directory. The <Compression Level> property can be used to - determine the Compression Level to use, from 0 (no compression) - to 9 (highest compression).
      • - -
      • FlowFileStream, v3 - This format would be used if the merged files - are to be sent to another NiFi system that is running NiFi version 3.0 or later. - When this format is selected, FlowFiles will be combined - into a FlowFileStream, which includes both FlowFile Attributes - and Content for all FlowFiles.
      • - -
      • FlowFileStream, v2 - This format would be used if the merged files - are to be sent to another NiFi system that is running a NiFi version older than NiFi-3.0. - When this format is selected, FlowFiles will be combined - into a FlowFileStream, which includes both FlowFile Attributes - and Content for all FlowFiles. Only use this if required for backward compatibility.
      • - -
      • FlowFile Tar, v1 - This format would be used if the merged files - are to be sent to another NiFi system that is running a much older NiFi version that does not support a FlowFileStream. - When this format is selected, FlowFiles will be combined - into a FlowFile Tar, which includes both FlowFile Attributes - and Content for all FlowFiles. Only use this if required for backward compatibility.
      • -
      -
    • Default value: Binary Concatenation
    • -
    • Supports expression language: false
    • -
    -
  • -
  • Attribute Strategy -
      -
    • Determines which FlowFile attributes should be added to - the bundle. If 'Keep All Unique Attributes' is selected, any - attribute on any FlowFile that gets bundled will be kept unless - its value conflicts with the value from another FlowFile. If - 'Keep Only Common Attributes' is selected, only the attributes - that exist on all FlowFiles in the bundle with the same value - will be preserved.
    • -
    • Default value: Keep Only Common Attributes
    • -
    • Supports expression language: false
    • -
  • -
  • Correlation Attribute Name -
      -
    • If specified, like FlowFiles will be binned together, - where 'like FlowFiles' means FlowFiles that have the same value - for this attribute. If not specified, and Merge Strategy is Defragment, - then the fragment.identifier attribute is automatically used. Otherwise, - FlowFiles are bundled by the order in which they are pulled from the queue.
    • -
    • Default value: no default
    • -
    • Supports expression language: false
    • -
    -
  • -
  • Minimum Number of Entries -
      -
    • The minimum number of files to include in a bundle.
    • -
    • Default value: 1
    • -
    • Supports expression language: false
    • -
  • -
  • Maximum Number of Entries -
      -
    • The maximum number of files to include in a bundle. If - not specified, there is no maximum.
    • -
    • Default value: no default
    • -
    • Supports expression language: false
    • -
    -
  • -
  • Minimum Group Size -
      -
    • The minimum size for each bundle.
    • -
    • Default value: 0 B
    • -
    • Supports expression language: false
    • -
  • -
  • Maximum Group Size -
      -
    • The maximum size for each bundle. If not specified, there - is no maximum.
    • -
    • Default value: no default
    • -
    • Supports expression language: false
    • -
    -
  • -
  • Max Bin Age -
      -
    • The maximum age of a bin that will trigger the bin to be - considered complete. The expected format for this value is a - positive integer and a time unit, such as seconds, minutes, - hours.
    • -
    • Default value: no default
    • -
    • Supports expression language: false
    • -
    -
  • -
  • Maximum Number of Bins -
      -
    • The maximum number of bins that can be held in memory at - any one time.
    • -
    • Default value: 100
    • -
    • Supports expression language: false
    • -
  • -
  • Header File -
      -
    • The local file path to a file that should be used as a header - on the merged file. If not supplied, no header is used. This property is - valid only when using Binary Concatenation as the Merge Format; - otherwise, it is ignored.
    • -
    • Default value: no default
    • -
    • Supports expression language: false
    • -
    -
  • -
  • Footer File -
      -
    • The local file path to a file that should be used as a footer - on the merged file. If not supplied, no footer is used. This property is - valid only when using Binary Concatenation as the Merge Format; - otherwise, it is ignored.
    • -
    • Default value: no default
    • -
    • Supports expression language: false
    • -
    -
  • -
  • Demarcator File -
      -
    • The local file path the file that specifies the - demarcator to use. If not specified, no demarcator is supplied. - This property is valid only when using Binary Concatenation as - the Merge Format; otherwise, it is ignored.
    • -
    • Default value: no default
    • -
    • Supports expression language: false
    • -
    -
  • -
  • Compression Level -
      -
    • Specifies the compression level to use when using the ZIP - merge format. If not using the zip merge format, this value is - ignored.
    • -
    • Default value: 1
    • -
    • Supports expression language: false
    • -
  • -
  • Keep Path -
      -
    • A Boolean value (true/false), indicating whether to keep - the FlowFiles' file paths in their entry names if using the ZIP - or TAR merge strategies. If not using one of those strategies, - this value is ignored.
    • -
    • Default value: false
    • -
    • Supports expression language: false
    • -
  • -
- -

- Relationships: -

-
    -
  • failure -
      -
    • If something prevents the processor from bundling the - incoming files, then they follow this relationship.
    • -
    -
  • -
  • merged -
      -
    • If the merging was successful, then the FlowFile - containing the merged content follows this relationship.
    • -
    -
  • -
  • original -
      -
    • If the merging was successful, then an original copy of - each file that was used to create a bundle follows this - relationship.
    • -
    -
  • -
- - diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.ModifyBytes/index.html b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.ModifyBytes/index.html deleted file mode 100644 index 285814c5c2..0000000000 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.ModifyBytes/index.html +++ /dev/null @@ -1,64 +0,0 @@ - - - - - - ModifyBytes - - - - - - -

Description:

-

- This processor updates the content of a FlowFile by removing bytes from start or end of a file. -

-

- Properties: -

-

- In the list below, the names of required properties appear in bold. Any other properties (not in bold) are considered optional. If a property has a default value, it is indicated. If a property supports the use of the NiFi Expression Language (or simply, "expression language"), that is also indicated. -

-
    -
  • Start Offset -
      -
    • Number of bytes removed at the beginning of the file. -
    • -
    • Default value: 0 B
    • -
    • Supports expression language: false
    • -
  • -
  • End Offset -
      -
    • Number of bytes removed at the end of the file. -
    • -
    • Default value: 0 B
    • -
    • Supports expression language: false
    • -
  • -
- -

- Relationships: -

-
    -
  • success -
      -
    • If FlowFiles are successfully updated, then they follow this relationship.
    • -
    -
  • -
- - diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.MonitorActivity/additionalDetails.html b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.MonitorActivity/additionalDetails.html new file mode 100644 index 0000000000..e05db70d4f --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.MonitorActivity/additionalDetails.html @@ -0,0 +1,49 @@ + + + + + + MonitorActivity + + + + + + + +

+ Modifies Attributes: +

+ + + + + + + + + + + + + + + + + +
Attribute NameDescription
inactivityStartMillisThe time at which Inactivity began, in the form of milliseconds since Epoch.
inactivityDurationMillisThe number of milliseconds that the inactivity has spanned.
+ + diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.MonitorActivity/index.html b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.MonitorActivity/index.html deleted file mode 100644 index d57c640a60..0000000000 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.MonitorActivity/index.html +++ /dev/null @@ -1,143 +0,0 @@ - - - - - - MonitorActivity - - - - - - -

Description:

-

This processor monitors its point in the dataflow for - activity and sends a notice when there is a lack of data flowing - through it for some user-specified amount of time; it then sends - another notice when the flow of data resumes.

- -

- Modifies Attributes: -

- - - - - - - - - - - - - - - - - -
Attribute NameDescription
inactivityStartMillisThe time at which Inactivity began, in the form of milliseconds since Epoch.
inactivityDurationMillisThe number of milliseconds that the inactivity has spanned.
- -

- Properties: -

-

In the list below, the names of required properties appear - in bold. Any other properties (not in bold) are considered optional. - If a property has a default value, it is indicated. If a property - supports the use of the NiFi Expression Language (or simply, - "expression language"), that is also indicated.

-
    -
  • Threshold Duration -
      -
    • The amount of time that must elapse before the flow is - considered inactive.
    • -
    • Default value: 5 min
    • -
    • Supports expression language: false
    • -
  • -
  • Continually Send Messages -
      -
    • A Boolean value (true/false), indicating whether to send - inactivity notices continually every time the threshold duration - period elapses until activity is restored. If false, the processor - will send a notice only when the flow first becomes inactive.
    • -
    • Default value: false
    • -
    • Supports expression language: false
    • -
  • -
  • Inactivity Message -
      -
    • The message that will appear as the content of outgoing - FlowFiles that are sent to the inactive relationship.
    • -
    • Default value: -
        -
      • Lacking activity as of time: ${now():format('yyyy/MM/dd - HH:mm:ss')}; flow has been inactive for - ${inactivityDurationMillis:toNumber():divide(60000)} minutes.
      • -
      -
    • -
    • Supports expression language: true
    • -
  • -
  • Activity Restored Message -
      -
    • The message that will appear as the content of outgoing - FlowFiles that are sent to the activity.restored relationship.
    • -
    • Default value: -
        -
      • Activity restored at time: ${now():format('yyyy/MM/dd - HH:mm:ss')} after being inactive for - ${inactivityDurationMillis:toNumber():divide(60000)} minutes.
      • -
      -
    • -
    • Supports expression language: true
    • -
  • -
  • Copy Attributes -
      -
    • If true, will copy all flow file attributes from the flow file that resumed activity to the newly created indicator flow file.
    • -
    • Default value: false -
    • -
    • Supports expression language: false
    • -
  • -
- -

- Relationships: -

-
    -
  • activity.restored -
      -
    • FlowFiles containing the content specified by the Activity - Restored Message property follow this relationship when FlowFiles - are routing to the success relationship following a period of - inactivity.
    • -
    -
  • -
  • inactive -
      -
    • FlowFiles containing the content specified by the - Inactivity Message property follow this relationship when FlowFiles - stop routing to the success relationship for the period of time set - in the Threshold Duration property.
    • -
    -
  • -
  • success -
      -
    • All incoming FlowFiles follow this relationship.
    • -
    -
  • -
- - - - diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.PostHTTP/additionalDetails.html b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.PostHTTP/additionalDetails.html new file mode 100644 index 0000000000..5ba60fac82 --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.PostHTTP/additionalDetails.html @@ -0,0 +1,48 @@ + + + + + + PostHTTP + + + + + + + + + +

+ Uses Attributes: +

+ + + + + + + + + + + + + +
Attribute NameDescription
mime.typeIf not sending data as a FlowFile, the mime.type attribute will be used to set the HTTP Header for Content-Type.
+ + + diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.PostHTTP/index.html b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.PostHTTP/index.html deleted file mode 100644 index b32a5869fa..0000000000 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.PostHTTP/index.html +++ /dev/null @@ -1,187 +0,0 @@ - - - - - - PostHTTP - - - - - - - - -

Description:

-

This processor performs an HTTP post with the content of - each incoming FlowFile. -

- -

- Uses Attributes: -

- - - - - - - - - - - - - -
Attribute NameDescription
mime.typeIf not sending data as a FlowFile, the mime.type attribute will be used to set the HTTP Header for Content-Type.
- -

- Properties: -

-

In the list below, the names of required properties appear - in bold. Any other properties (not in bold) are considered optional. - If a property has a default value, it is indicated. If a property - supports the use of the NiFi Expression Language (or simply, - "expression language"), that is also indicated.

-
    -
  • URL -
      -
    • The URL to post to. The first part of the URL must be - static. However, the path of the URL may be defined using - NiFi Expression Language. For example, https://${hostname} - is not valid, but https://1.1.1.1:8080/files/${nf.file.name} is - valid.
    • -
    • Default value: no default
    • -
    • Supports expression language: true
    • -
  • -
  • Max Batch Size -
      -
    • Specifies the max data size for a batch of FlowFiles to - send in a single HTTP POST. If nothing is specified, each FlowFile - will be sent separately.
    • -
    • Default value: 100 MB
    • -
    • Supports expression language: false
    • -
    -
  • -
  • Max Data To Post per Second -
      -
    • The maximum amount of data to send per second. This allows - the bandwidth to be throttled to a specified data rate. If not - specified, the data rate is not throttled.
    • -
    • Default value: no default
    • -
    • Supports expression language: false
    • -
    -
  • -
  • SSL Context Service -
      -
    • The Controller Service to use for obtaining an SSL Context. The SSL Context controller service is a mechanism for providing all the - security properties that allow for secure communications between NiFi extensions and other systems. See the User Guide or the Controller Services documentation - via the "help" link in the upper-right corner of the GUI for more information about the StandardSSLContextService. The value for this property is the - identifier name that is configured in the StandardSSLContextService.
    • -
    • Default value: no default
    • -
    • Supports expression language: false
    • -
    -
  • -
  • Username -
      -
    • The username required to access the URL.
    • -
    • Default value: no default
    • -
    • Supports expression language: false
    • -
    -
  • -
  • Password -
      -
    • The password required to access the URL.
    • -
    • Default value: no default
    • -
    • Supports expression language: false
    • -
    -
  • -
  • Send as FlowFile -
      -
    • A Boolean value (true/false), indicating whether to package - the FlowFile's contents and attributes together and send them as a - FlowFile package; otherwise, it will send only the FlowFile's - content.
    • -
    • Default value: false
    • -
    • Supports expression language: false
    • -
  • -
  • Use Chunked Encoding -
      -
    • Specifies whether to use chunked encoding to send the data. If false, the entire - content of the FlowFile will be buffered into memory. The default is true.
    • -
    • Default value: true
    • -
    • Supports expression language: false
    • -
  • -
  • Compression Level -
      -
    • Specifies the GZIP compression level to use when sending - the file. The value must be within the range of 0-9. A value of 0 - indicates that the file should not be GZIP'd.
    • -
    • Default value: 0
    • -
    • Supports expression language: false
    • -
  • -
  • Connection Timeout -
      -
    • The amount of time to wait before timing out while creating - a connection.
    • -
    • Default value: 30 sec
    • -
    • Supports expression language: false
    • -
  • -
  • Data Timeout -
      -
    • The amount of time to wait before timing out while - transferring data.
    • -
    • Default value: 30 sec
    • -
    • Supports expression language: false
    • -
  • -
  • Attributes to Send as HTTP Headers (Regex) -
      -
    • Specifies the regular expression that determines the names - of FlowFile attributes that should be sent as HTTP headers.
    • -
    • Default value: no default
    • -
    • Supports expression language: false
    • -
    -
  • -
  • User Agent -
      -
    • What to report as the user agent when a connection is made - to the remote server.
    • -
    • Default value: no default
    • -
    • Supports expression language: false
    • -
    -
  • -
- -

- Relationships: -

-
    -
  • failure -
      -
    • If something prevents the processor from successfully - posting the FlowFile, then the FlowFile follows this relationship.
    • -
    -
  • -
  • success -
      -
    • If the processor successfully posts the FlowFile, then it - follows this relationship.
    • -
    -
  • -
- - - diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.PutEmail/index.html b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.PutEmail/index.html deleted file mode 100644 index 1160dbafc3..0000000000 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.PutEmail/index.html +++ /dev/null @@ -1,114 +0,0 @@ - - - - - - PutEmail - - - - - - -

Description:

-

- This processor sends an e-mail to configured recipients for each incoming FlowFile.

-

- Properties: -

-

- In the list below, the names of required properties appear in bold. Any other properties (not in bold) are considered optional. If a property has a default value, it is indicated. If a property supports the use of the NiFi Expression Language (or simply, "expression language"), that is also indicated. -

-
    -
  • SMTP Hostname -
      -
    • The hostname of the SMTP host.
    • -
    • Default value: no default
    • -
    • Supports expression language: false
    • -
  • -
  • SMTP Port -
      -
    • The port used for SMTP communication.
    • -
    • Default value: 25
    • -
    • Supports expression language: false
    • -
  • -
  • From -
      -
    • Specifies the email address to use as the sender.
    • -
    • Default value: no default
    • -
    • Supports expression language: true
    • -
  • -
  • To -
      -
    • The recipients to include in the To line of the email.
    • -
    • Default value: no default
    • -
    • Supports expression language: true
    • -
  • -
  • CC -
      -
    • The recipients to include in the CC line of the email.
    • -
    • Default value: no default
    • -
    • Supports expression language: true
    • -
  • -
  • BCC -
      -
    • The recipients to include in the BCC line of the email.
    • -
    • Default value: no default
    • -
    • Supports expression language: true
    • -
  • -
  • Subject -
      -
    • This content will appear in the email Subject line.
    • -
    • Default value: Message from NiFi
    • -
    • Supports expression language: true
    • -
  • -
  • Message -
      -
    • The body of the email message.
    • -
    • Default value: no default
    • -
    • Supports expression language: true
    • -
  • -
  • Attach File -
      -
    • A Boolean value (true/false), indicating whether to attach the FlowFile content to the email.
    • -
    • Default value: false
    • -
    • Supports expression language: false
    • -
  • -
  • Include All Attributes In Message -
      -
    • A Boolean value (true/false), indicating whether or not all FlowFile attributes should be recorded - in the body of the email message.
    • -
    • Default value: false
    • -
    • Supports expression language: false
    • -
  • -
-

- Relationships: -

-
    -
  • failure -
      -
    • If FlowFiles fail to send, then they follow this relationship.
    • -
  • -
  • success -
      -
    • If an email is successfully sent for a given FlowFile, then the FlowFile follows this relationship.
    • -
  • -
- - - - diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.PutFTP/additionalDetails.html b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.PutFTP/additionalDetails.html new file mode 100644 index 0000000000..fbece63817 --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.PutFTP/additionalDetails.html @@ -0,0 +1,66 @@ + + + + + + PutFTP + + + + + + +

+ Optional User-Defined Properties: +

+
    +
  • pre.cmd.# +
      +
    • Optional properties of this type can be added by the user and should be used for VERY + RARE cases only. You will know when you need to use it when the recipient specifies that you must + send FTP commands prior to the transfer. Otherwise, do not use it. You may add these optional properties + to send any commands to the FTP server before the file is actually transferred (before the put command). + This option is only available for the PutFTP processor, as only FTP has this functionality. This is + essentially the same as sending quote commands to an FTP server from the command line. +
    • +
    • NOTE: While this is the same as sending a quote command, it is very important that + you leave off the “quote” part of the command from the text value you enter in. For example, if you want + to mimic the command line ftp command quote stat you would give the text value of stat. You have the + option of sending a number of commands to the FTP server before the file is sent. If you have only one + command, then the name of the optional parameter will be pre.cmd.1. If you have a second command, then + the next optional parameter you give is pre.cmd.2, and so on. There is no limit to the number of pre + commands you can send. The commands will be sent in the order of their number, so pre.cmd.1 will be sent + first, pre.cmd.2 will be sent next, etc. This is true regardless of what order they are added to the + processor properties. The property names determine the order.
    • +
    • NOTE: If the optional property name does not follow the pattern pre.cmd.integer + - then the command will not be sent. If a command is given which is not recognized by the server, then + that will be logged.
    • +
    • Supports expression language: true. FlowFile attributes can be used in commands using the expression language
    • +
    +
  • +
  • post.cmd.# +
      +
    • Optional properties of this type can be added by the user and should be used for VERY + RARE cases only. You will know when you need to use it when the recipient specifies that you must + send FTP commands after the transfer. Otherwise, do not use it. These are the same as the pre commands. + (See pre commands above) except that these commands are sent after the file transfer. +
    • +
    • Supports expression language: true. FlowFile attributes can be used in commands using the expression language
    • +
    +
  • +
+ + diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.PutFTP/index.html b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.PutFTP/index.html deleted file mode 100644 index 2edee6a4a1..0000000000 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.PutFTP/index.html +++ /dev/null @@ -1,283 +0,0 @@ - - - - - - PutFTP - - - - - -

Description:

-

- This processor sends FlowFiles via FTP to an FTP server. -

-

- Properties: -

-

- In the list below, the names of required properties appear in bold. - Any other properties (not in bold) are considered optional. If a - property has a default value, it is indicated. If a property supports - the use of the NiFi Expression Language (or simply, - "expression language"), that is also indicated. -

-
    -
  • - Hostname -
      -
    • The fully qualified hostname or IP address of the remote - system.
    • -
    • Default value: no default
    • -
    • Supports expression language: true
    • -
    -
  • -
  • - Port -
      -
    • The port that the remote system is listening on for file transfers.
    • -
    • Default value: 21
    • -
    • Supports expression language: false
    • -
    -
  • -
  • - Username -
      -
    • The username for the user account.
    • -
    • Default value: no default
    • -
    • Supports expression language: false
    • -
    -
  • -
  • - Password -
      -
    • The password for the user account.
    • -
    • Default value: no default
    • -
    • Supports expression language: false
    • -
    -
  • -
  • - Remote Path -
      -
    • The path on the remote system to which files should be put. If not specified, uses the user's home directory. You may use expression language such as /aa/bb/${path}.
    • -
    • Default value: no default
    • -
    • Supports expression language: true
    • -
    -
  • -
  • Create Directory -
      -
    • Specifies whether or not the remote directory should be created if it does not exist. Recommend setting to true when Remote Path uses expression language.
    • -
    • Default value: false
    • -
    • Supports expression language: false
    • -
    -
  • -
  • - Batch Size -
      -
    • The maximum number of FlowFiles to send in a single connection.
    • -
    • Default value: 500
    • -
    • Supports expression language: false
    • -
    -
  • -
  • - Connection Timeout -
      -
    • The amount of time to wait before timing out while creating a - connection.
    • -
    • Default value: 30 sec
    • -
    • Supports expression language: false
    • -
    -
  • -
  • - Data Timeout -
      -
    • The amount of time to wait before timing out while transferring - data.
    • -
    • Default value: 30 sec
    • -
    • Supports expression language: false
    • -
    -
  • -
  • - Conflict Resolution -
      -
    • - Specifies what action the processor should take if a conflict - prevents it from delivering the files. Valid options are: -
        -
      • replace
      • -
      • ignore
      • -
      • rename
      • -
      • reject
      • -
      • fail
      • -
      • NONE
      • -
      -
    • -
    • Default value: NONE
    • -
    • Supports expression language: false
    • -
    -
  • -
  • - Dot Rename -
      -
    • A Boolean value (true/false), indicating whether to prepend the - filename of the file with a dot (.) while the file is transferring - and remove the dot when the file is completely transferred.
    • -
    • Default value: true
    • -
    • Supports expression language: false
    • -
    -
  • -
  • - Temporary Filename -
      -
    • If set, the filename of the sent file will be equal to the value specified during the transfer and - after successful completion will be renamed to the original filename. If this value is set, the Dot - Rename property is ignored. -
    • -
    • Default value: no default
    • -
    • Supports expression language: true
    • -
    -
  • -
  • - Transfer Mode -
      -
    • - The FTP transfer mode. Valid options are: Binary or ASCII. -
    • -
    • Default value: Binary
    • -
    • Supports expression language: false
    • -
    -
  • -
  • - Connection Mode -
      -
    • - The FTP connection mode. Valid options are: Active or Passive. - For most clients, this should be set to Passive.
    • -
    • Default value: Passive
    • -
    • Supports expression language: false
    • -
    -
  • -
  • - Reject Zero-Byte Files -
      -
    • A Boolean value (true/false), indicating whether to reject - files that have zero bytes of content rather than transferring - them.
    • -
    • Default value: true
    • -
    • Supports expression language: false
    • -
    -
  • -
  • Last Modified Time -
      -
    • The lastModifiedTime to assign to the file after transferring it. If not set, the lastModifiedTime - will not be changed. Format must be yyyy-MM-dd'T'HH:mm:ssZ. You may also use expression language such as - ${file.lastModifiedTime}. If the value is invalid, the processor will not be invalid but will fail to - change lastModifiedTime of the file.
    • -
    • Default value: no default
    • -
    • Supports expression language: true
    • -
    -
  • -
  • Permissions -
      -
    • The permissions to assign to the file after transferring it. Format must be either UNIX rwxrwxrwx - with a - in place of denied permissions (e.g. rw-r--r--) or an octal number (e.g. 644). If not set, the - permissions will not be changed. You may also use expression language such as ${file.permissions}. If - the value is invalid, the processor will not be invalid but will fail to change permissions of the file. -
    • -
    • Default value: no default
    • -
    • Supports expression language: true
    • -
    -
  • -
  • - Use Compression -
      -
    • A Boolean value (true/false), indicating whether to use ZLIB - compression when transferring files.
    • -
    • Default value: false
    • -
    • Supports expression language: false
    • -
    -
  • -
-

- Optional User-Defined Properties: -

-

-

    -
  • pre.cmd.# -
      -
    • Optional properties of this type can be added by the user and should be used for VERY - RARE cases only. You will know when you need to use it when the recipient specifies that you must - send FTP commands prior to the transfer. Otherwise, do not use it. You may add these optional properties - to send any commands to the FTP server before the file is actually transferred (before the put command). - This option is only available for the PutFTP processor, as only FTP has this functionality. This is - essentially the same as sending quote commands to an FTP server from the command line. -
    • -
    • NOTE: While this is the same as sending a quote command, it is very important that - you leave off the “quote†part of the command from the text value you enter in. For example, if you want - to mimic the command line ftp command quote stat you would give the text value of stat. You have the - option of sending a number of commands to the FTP server before the file is sent. If you have only one - command, then the name of the optional parameter will be pre.cmd.1. If you have a second command, then - the next optional parameter you give is pre.cmd.2, and so on. There is no limit to the number of pre - commands you can send. The commands will be sent in the order of their number, so pre.cmd.1 will be sent - first, pre.cmd.2 will be sent next, etc. This is true regardless of what order they are added to the - processor properties. The property names determine the order.
    • -
    • NOTE: If the optional property name does not follow the pattern pre.cmd.integer - - then the command will not be sent. If a command is given which is not recognized by the server, then - that will be logged.
    • -
    • Supports expression language: true. FlowFile attributes can be used in commands using the expression language
    • -
    -
  • -
  • post.cmd.# -
      -
    • Optional properties of this type can be added by the user and should be used for VERY - RARE cases only. You will know when you need to use it when the recipient specifies that you must - send FTP commands after the transfer. Otherwise, do not use it. These are the same as the pre commands. - (See pre commands above) except that these commands are sent after the file transfer. -
    • -
    • Supports expression language: true. FlowFile attributes can be used in commands using the expression language
    • -
    -
  • -
-

- -

- Relationships: -

-
    -
  • - failure -
      -
    • If something prevents a FlowFile from being transferred, then - it follows this relationship.
    • -
    -
  • -
  • - reject -
      -
    • FlowFiles that were rejected by the destination system follow this relationship.
    • -
    -
  • -
  • - success -
      -
    • If a FlowFile is successfully transferred, then it follows this - relationship.
    • -
    -
  • -
- - diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.PutFile/index.html b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.PutFile/index.html deleted file mode 100644 index bdb40d67b0..0000000000 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.PutFile/index.html +++ /dev/null @@ -1,109 +0,0 @@ - - - - - - PutFile - - - - - - -

Description:

-

- This processor writes FlowFiles to the local file system.

-

- Properties: -

-

- In the list below, the names of required properties appear in bold. Any other properties (not in bold) are considered optional. If a property has a default value, it is indicated. If a property supports the use of the NiFi Expression Language (or simply, "expression language"), that is also indicated. -

-
    -
  • Directory -
      -
    • The local file path to the location where files should be written. You may use expression language such as /aa/bb/${path}.
    • -
    • Default value: no default
    • -
    • Supports expression language: true
    • -
  • -
  • Keep Directory Structure -
      -
    • A Boolean value (true/false), indicating whether to maintain the file structure and write FlowFiles to subdirectories of the file path defined in the Directory property. - Note: This property is maintained for backward compatibility purposes but is considered "Deprecated" and should not be used (value should remain 'false'). - Rather than using this property, if the goal is to append the FlowFile's path to the /aa/bb directory, for example, the Directory property should - instead be set to /aa/bb/${path}. -
    • -
    • Default value: false
    • -
    • Supports expression language: false
    • -
  • -
  • Conflict Resolution Strategy -
      -
    • Indicates what should happen if the processor cannot write a file to the file path defined in the Directory property. This could happen, for example, if a file of the same name already exists in the directory. Valid options can be selected from the drop-down menu and include: -
        -
      • replace
      • -
      • ignore
      • -
      • fail
      • -
    • -
    • Default value: fail
    • -
    • Supports expression language: false
    • -
  • -
  • Maximum File Count -
      -
    • The maximum number of files that can exist in the output directory.
    • -
    • Default value: no default
    • -
    • Supports expression language: false
    • -
  • -
  • Last Modified Time -
      -
    • Sets the lastModifiedTime on the output file to the value of this attribute. Format must be yyyy-MM-dd'T'HH:mm:ssZ. You may also use expression language such as ${file.lastModifiedTime}.
    • -
    • Default value: no default
    • -
    • Supports expression language: true
    • -
  • -
  • Permissions -
      -
    • Sets the read/write/execute permissions on the output file to the value of this attribute. Format must be either UNIX rwxrwxrwx with a - in place of denied permissions (e.g. rw-r--r--) or an octal number (e.g. 644). You may also use expression language such as ${file.permissions}.
    • -
    • Default value: no default
    • -
    • Supports expression language: true
    • -
  • -
  • Owner -
      -
    • Sets the owner on the output file to the value of this attribute. You may also use expression language such as ${file.owner}. Normal access rules apply, meaning your system may not let you change file ownership.
    • -
    • Default value: no default
    • -
    • Supports expression language: true
    • -
  • -
  • Group -
      -
    • Sets the group on the output file to the value of this attribute. You may also use expression language such as ${file.group}. Normal access rules apply, meaning your system may not let you change group ownership.
    • -
    • Default value: no default
    • -
    • Supports expression language: true
    • -
  • -
-

- Relationships: -

-
    -
  • failure -
      -
    • If something prevents a FlowFile from being written to its destination directory, then it follows this relationship.
    • -
  • -
  • success -
      -
    • If a FlowFile is successfully written to its destination directory, then it follows this relationship.
    • -
  • -
- - - diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.PutJMS/index.html b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.PutJMS/index.html deleted file mode 100644 index 387d03c9b5..0000000000 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.PutJMS/index.html +++ /dev/null @@ -1,152 +0,0 @@ - - - - - - PutJMS - - - - - - -

Description:

-

- This processor creates a JMS message from the contents of a FlowFile and sends the message to a JMS server.

-

- Properties: -

-

- In the list below, the names of required properties appear in bold. Any other properties (not in bold) are considered optional. If a property has a default value, it is indicated. If a property supports the use of the NiFi Expression Language (or simply, "expression language"), that is also indicated. -

-
    -
  • JMS Provider -
      -
    • This property specifies the provider used for the JMS server. Available options include: -
        -
      • ActiveMQ
      • -
    • -
    • Default value: ActiveMQ
    • -
    • Supports expression language: false
    • -
  • -
  • URL -
      -
    • The URL of the JMS server.
    • -
    • Default value: no default
    • -
    • Supports expression language: false
    • -
  • -
  • Destination Name -
      -
    • The name of the JMS topic of queue to use.
    • -
    • Default value: no default
    • -
    • Supports expression language: false
    • -
  • -
  • Destination Type -
      -
    • The type of JMS Destination to use. Options are Queue and Topic.
    • -
    • Default value: Queue
    • -
    • Supports expression language: false
    • -
  • -
  • Communications Timeout -
      -
    • The amount of time to wait when attempting to receive a message before giving up and assuming failure.
    • -
    • Default value: 30 sec
    • -
    • Supports expression language: false
    • -
  • -
  • Batch Size -
      -
    • The number of messages to Put in a single iteration of the processor
    • -
    • Default value: 10
    • -
    • Supports expression language: false
    • -
  • -
  • Username -
      -
    • The username used for authentication and authorization.
    • -
    • Default value: no default
    • -
    • Supports expression language: false
    • -
  • -
  • Password -
      -
    • The password used for authentication and authorization.
    • -
    • Default value: no default
    • -
    • Supports expression language: false
    • -
  • -
  • Message Type -
      -
    • The type of JMS message to construct. Available options include: -
        -
      • byte
      • -
      • stream
      • -
      • text
      • -
      • empty
      • -
    • -
    • Default value: byte
    • -
    • Supports expression language: false
    • -
  • -
  • Message Priority -
      -
    • The priority of the message.
    • -
    • Default value: no default
    • -
    • Supports expression language: true
    • -
  • -
  • Reply-to Queue -
      -
    • The name of the queue to which a reply should be added.
    • -
    • Default value: no default
    • -
    • Supports expression language: true
    • -
  • -
  • Max Buffer Size -
      -
    • The maximum amount of data that can be buffered for a JMS message. If a FlowFile's size exceeds this value, it will be routed to failure.
    • -
    • Default value: 1 MB
    • -
    • Supports expression language: false
    • -
  • -
  • Message Time to Live -
      -
    • The amount of time that the message should live on the destination before being removed; if not specified, the message will never expire.
    • -
    • Default value: no default
    • -
    • Supports expression language: false
    • -
  • -
  • Copy Attributes to JMS Properties -
      -
    • A Boolean value (true/false), indicating whether FlowFile attributes should be translated into JMS Message Properties. If true, all attributes starting with 'jms.' will be set as properties on the JMS message (without the 'jms.' prefix). If an attribute exists that starts with the same value but ends in '.type', that attribute will be used to determine the JMS Message Property type.
    • -
    • Default value: true
    • -
    • Supports expression language: false
    • -
  • -
  • Client ID Prefix -
      -
    • A human-readable ID that can be used to associate connections with yourself so that the maintainers of the JMS Server know who to contact if problems arise.
    • -
    • Default value: no default
    • -
    • Supports expression language: false
    • -
  • -
-

- Relationships: -

-
    -
  • failure -
      -
    • If something prevents a FlowFile from being routed to the JMS destination, then it follows this relationship.
    • -
  • -
  • success -
      -
    • If a FlowFile is successfully routed to the JMS destination, then it follows this relationship.
    • -
  • -
- - - - diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.PutSFTP/additionalDetails.html b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.PutSFTP/additionalDetails.html new file mode 100644 index 0000000000..656e150ebc --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.PutSFTP/additionalDetails.html @@ -0,0 +1,34 @@ + + + + + + PutSFTP + + + + + +

+ See Also: +

+ + + diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.PutSFTP/index.html b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.PutSFTP/index.html deleted file mode 100644 index 8f30e7beb2..0000000000 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.PutSFTP/index.html +++ /dev/null @@ -1,281 +0,0 @@ - - - - - - PutSFTP - - - - - -

Description:

-

- This processor sends FlowFiles via SFTP to an SFTP server. -

-

- Properties: -

-

- In the list below, the names of required properties appear in bold. - Any other properties (not in bold) are considered optional. If a - property has a default value, it is indicated. If a property supports - the use of the NiFi Expression Language (or simply, - "expression language"), that is also indicated. -

-
    -
  • - Hostname -
      -
    • The fully qualified hostname or IP address of the remote - system.
    • -
    • Default value: no default
    • -
    • Supports expression language: true
    • -
    -
  • -
  • - Port -
      -
    • The port that the remote system is listening on for file transfers.
    • -
    • Default value: 22
    • -
    • Supports expression language: false
    • -
    -
  • -
  • - Username -
      -
    • The username for the user account.
    • -
    • Default value: no default
    • -
    • Supports expression language: false
    • -
    -
  • -
  • - Password -
      -
    • The password for the user account.
    • -
    • Default value: no default
    • -
    • Supports expression language: false
    • -
    -
  • -
  • - Private Key Path -
      -
    • The fully qualified path for the private key file.
    • -
    • Default value: no default
    • -
    • Supports expression language: false
    • -
    -
  • -
  • - Private Key Passphrase -
      -
    • The password for the private key.
    • -
    • Default value: no default
    • -
    • Supports expression language: false
    • -
    -
  • -
  • - Remote Path -
      -
    • The path on the remote system to which files should be transferred. If not specified, the user's home directory will be used. You may use expression language such as /aa/bb/${path}.
    • -
    • Default value: no default
    • -
    • Supports expression language: true
    • -
    -
  • -
  • Create Directory -
      -
    • Specifies whether or not the remote directory should be created if it does not exist. Recommend setting to true when Remote Path uses expression language.
    • -
    • Default value: false
    • -
    • Supports expression language: false
    • -
    -
  • -
  • Batch Size -
      -
    • The maximum number of FlowFiles to send in a single connection.
    • -
    • Default value: 500
    • -
    • Supports expression language: false
    • -
    -
  • -
  • - Connection Timeout -
      -
    • The amount of time to wait before timing out while creating a - connection.
    • -
    • Default value: 30 sec
    • -
    • Supports expression language: false
    • -
    -
  • -
  • - Data Timeout -
      -
    • The amount of time to wait before timing out while transferring - data.
    • -
    • Default value: 30 sec
    • -
    • Supports expression language: false
    • -
    -
  • -
  • - Conflict Resolution -
      -
    • Specifies what action the processor should take if a conflict - prevents it from delivering the files. Valid options are:
    • -
        -
      • replace
      • -
      • ignore
      • -
      • rename
      • -
      • reject
      • -
      • fail
      • -
      • NONE
      • -
      -
    • Default value: NONE
    • -
    • Supports expression language: false
    • -
    -
  • -
  • - Reject Zero-Byte Files -
      -
    • A Boolean value (true/false), indicating whether to reject - files that have zero bytes of content rather than transferring - them.
    • -
    • Default value: true
    • -
    • Supports expression language: false
    • -
    -
  • -
  • - Dot Rename -
      -
    • A Boolean value (true/false), indicating whether to prepend the - filename of the file with a dot (.) while the file is transferring - and remove the dot when the file is completely transferred.
    • -
    • Default value: true
    • -
    • Supports expression language: false
    • -
    -
  • -
  • Temporary Filename -
      -
    • If set, the filename of the sent file will be equal to the value specified during the transfer and - after successful completion will be renamed to the original filename. If this value is set, the Dot - Rename property is ignored. -
    • -
    • Default value: no default
    • -
    • Supports expression language: true
    • -
    -
  • -
  • - Host Key File -
      -
    • The local file path to the host key file; if not supplied, no - host key file will be used.
    • -
    • Default value: no default
    • -
    • Supports expression language: false
    • -
    -
  • -
  • Last Modified Time -
      -
    • The lastModifiedTime to assign to the file after transferring it. If not set, the lastModifiedTime - will not be changed. Format must be yyyy-MM-dd'T'HH:mm:ssZ. You may also use expression language such as - ${file.lastModifiedTime}. If the value is invalid, the processor will not be invalid but will fail to - change lastModifiedTime of the file.
    • -
    • Default value: no default
    • -
    • Supports expression language: true
    • -
    -
  • -
  • Permissions -
      -
    • The permissions to assign to the file after transferring it. Format must be either UNIX rwxrwxrwx - with a - in place of denied permissions (e.g. rw-r--r--) or an octal number (e.g. 644). If not set, the - permissions will not be changed. You may also use expression language such as ${file.permissions}. If - the value is invalid, the processor will not be invalid but will fail to change permissions of the file. -
    • -
    • Default value: no default
    • -
    • Supports expression language: true
    • -
    -
  • -
  • Remote Owner -
      -
    • Integer value representing the User ID to set on the file after transferring it. If not set, the - owner will not be changed. You may also use expression language such as ${file.owner}. If the value is - invalid, the processor will not be invalid but will fail to change the owner of the file.
    • -
    • Default value: no default
    • -
    • Supports expression language: true
    • -
    -
  • -
  • Remote Group -
      -
    • Integer value representing the Group ID to set on the file after transferring it. If not set, the - group will not be changed. You may also use expression language such as ${file.group}. If the value is - invalid, the processor will not be invalid but will fail to change the group of the file.
    • -
    • Default value: no default
    • -
    • Supports expression language: true
    • -
    -
  • -
  • - Strict Host Key Checking -
      -
    • A Boolean value (true/false), indicating whether to apply - strict enforcement of host keys.
    • -
    • Default value: false
    • -
    • Supports expression language: false
    • -
    -
  • -
  • - Use Compression -
      -
    • A Boolean value (true/false), indicating whether to use ZLIB - compression when transferring files.
    • -
    • Default value: false
    • -
    • Supports expression language: false
    • -
    -
  • -
- -

- Relationships: -

-
    -
  • - failure -
      -
    • If FlowFiles fail to be transferred, then they follow this - relationship.
    • -
    -
  • -
  • - reject -
      -
    • If the property Reject Zero-Byte Files is set to true and - incoming FlowFiles meet this criteria, then they follow this - relationship.
    • -
    -
  • -
  • - success -
      -
    • If FlowFiles are successfully transferred, then they follow - this relationship.
    • -
    -
  • -
- -

- See Also: -

-

- - diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.ReplaceText/index.html b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.ReplaceText/index.html deleted file mode 100644 index ffcf205e02..0000000000 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.ReplaceText/index.html +++ /dev/null @@ -1,91 +0,0 @@ - - - - - - ReplaceText - - - - - - -

Description:

-

- This processor updates the content of a FlowFile by evaluating a regular expression (regex) against the content and replacing the section of content that matches the regular expression with an alternate, user-defined, value. -

-

- Properties: -

-

- In the list below, the names of required properties appear in bold. Any other properties (not in bold) are considered optional. If a property has a default value, it is indicated. If a property supports the use of the NiFi Expression Language (or simply, "expression language"), that is also indicated. -

-
    -
  • Regular Expression -
      -
    • The regular expression to use to evaluate the FlowFile content. While the Expression Language may be used here, the values returned by the Expression Language will be escaped. I.e., they will not be interpreted as Regular Expressions but will be interpreted as literal values. - For example, if you have an attribute named X with the value H.*o and want it to match against "Hello", you cannot do this by referencing ${X}. Rather, a reference to ${X} - will reference the exact literal H.*o -
    • -
    • Default value: (.*)
    • -
    • Supports expression language: true
    • -
  • -
  • Replacement Value -
      -
    • The value to replace matching content with. Back-references to Regular Expression capturing groups are supported, but back-references that reference capturing groups that do not exist in the regular expression are treated as literal values.
    • -
    • Default value: $1
    • -
    • Supports expression language: true
    • -
  • -
  • Character Set -
      -
    • The character set in which the file is encoded.
    • -
    • Default value: UTF-8
    • -
    • Supports expression language: false
    • -
  • -
  • Evaluation Mode -
      -
    • Evaluate the 'Regular Expression' against each line (Line-by-Line) or buffer the entire file into memory (Entire Text) and then evaluate the 'Regular Expression'.
    • -
    • Default value: Entire text
    • -
    • Supports expression language: false
    • -
  • -
  • Maximum Buffer Size -
      -
    • Specifies the maximum amount of data to buffer (per file or per line, depending on the Evaluation Mode) in order to apply the regular expressions. If 'Entire Text' Mode is selected and the FlowFile is larger than this value, the FlowFile will be routed to 'failure'. - In 'Line-by-Line' Mode, if a single line is larger than this value, the FlowFile will be routed to 'failure'. A default value of 1 MB is provided, primarily for 'Entire Text' mode. In 'Line-by-Line' Mode, a value such as 8 KB or 16 KB is suggested. - This value is ignored and the buffer is not used if 'Regular Expression' is set to '.*' -
    • -
    • Default value: 1MB
    • -
    • Supports expression language: false
    • -
  • -
- -

- Relationships: -

-
    -
  • failure -
      -
    • If FlowFiles cannot be updated, then they follow this relationship.
    • -
  • -
  • success -
      -
    • If FlowFiles are successfully updated, then they follow this relationship.
    • -
  • -
- - - - diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.ReplaceTextWithMapping/index.html b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.ReplaceTextWithMapping/index.html deleted file mode 100644 index 1d2eb2cd00..0000000000 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.ReplaceTextWithMapping/index.html +++ /dev/null @@ -1,114 +0,0 @@ - - - - - ReplaceText - - - - - - -

Description:

-

- This processor updates the content of a FlowFile by evaluating a - regular - expression (regex) against the content and replacing the section of - content that matches a specified matching  group of the - regular expression with an alternate, - user-defined, value provided in a mapping file.  The mapping - file is formatted as one key/value pair per line, seperated by tabs. -

-

- Properties: -

-

- In the list below, the names of required properties appear in bold. - Any other properties (not in bold) are considered optional. If a - property has a default value, it is indicated. If a property supports - the use of the NiFi Expression Language (or simply, "expression - language"), that is also indicated. -

-
    -
  • Regular Expression -
      -
    • The regular expression to use to evaluate the FlowFile - content.
    • -
    • Default value: (\\S+)
    • -
    • Supports expression language: true
    • -
    -
  • -
  • Matching Group -
      -
    • The number of the matching group of the provided regex - to replace - with the corresponding value from the mapping file (if it exists).
    • -
    • Default value: 0
    • -
    • Supports expression language: true
    • -
    -
  • -
  • Mapping File -
      -
    • The name of the file (including the full path) - containing the Mappings.
    • -
    • Default value: no default
    • -
    • Supports expression language: false
    • -
    -
  • -
  • Mapping File Refresh Interval -
      -
    • The polling interval in seconds to check for updates to - the mapping file.
    • -
    • Default value: 60s
    • -
    • Supports expression language: false
    • -
    -
  • -
  • Character Set -
      -
    • The character set in which the file is encoded.
    • -
    • Default value: UTF-8
    • -
    • Supports expression language: false
    • -
    -
  • -
  • Maximum Buffer Size -
      -
    • Specifies the maximum amount of data to buffer (per - file) in order to apply the regular expression. If a FlowFile is larger - than this value, then the FlowFile is routed to the failure - relationship.
    • -
    • Default value: 1MB
    • -
    • Supports expression language: false
    • -
    -
  • -
-

- Relationships: -

-
    -
  • failure -
      -
    • If FlowFiles cannot be updated, then they follow this - relationship.
    • -
    -
  • -
  • success -
      -
    • If FlowFiles are successfully updated, then they follow - this relationship.
    • -
    -
  • -
- diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.RouteOnAttribute/additionalDetails.html b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.RouteOnAttribute/additionalDetails.html new file mode 100644 index 0000000000..9bd3c383f9 --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.RouteOnAttribute/additionalDetails.html @@ -0,0 +1,46 @@ + + + + + + RouteOnAttribute + + + + + +

Description:

+

+ This processor routes FlowFiles based on their attributes + using the NiFi Expression Language. Users add properties with + valid NiFi Expression Language Expressions as the values. Each Expression must + return a value of type Boolean (true or false). +

+

+ Example: The goal is to route all files with filenames that start with ABC down a certain path. + Add a property with the following name and value: +

+
    +
  • property name: ABC +
  • +
  • property value: ${filename:startsWith('ABC')} +
  • +
+

+ In this example, all files with filenames that start with ABC will follow the ABC relationship. +

+ + diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.RouteOnAttribute/index.html b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.RouteOnAttribute/index.html deleted file mode 100644 index 5b7f820bb7..0000000000 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.RouteOnAttribute/index.html +++ /dev/null @@ -1,110 +0,0 @@ - - - - - - RouteOnAttribute - - - - - -

Description:

-

- This processor routes FlowFiles based on their attributes - using the NiFi Expression Language. Users add properties with - valid NiFi Expression Language Expressions as the values. Each Expression must - return a value of type Boolean (true or false). -

-

- Example: The goal is to route all files with filenames that start with ABC down a certain path. - Add a property with the following name and value: -

-
    -
  • property name: ABC -
  • -
  • property value: ${filename:startsWith('ABC')} -
  • -
-

- In this example, all files with filenames that start with ABC will follow the ABC relationship. -

- -

- Properties: -

-

- In the list below, the names of required properties appear - in bold. -

- -
    -
  • Routing Strategy -
      -
    • Specifies how to determine which relationship to use when - evaluating the expression language. Options are: -
        -
      • Route to Property Name - For each property that has been added, if its Expression evaluates to true - for a given FlowFile, that FlowFile will be routed to the Relationship whose name is defined by the property name. - If multiple properties' Expressions evaluate to true, a clone of the FlowFile will be created - for each relationship.
      • - -
      • Route to 'match' if all match - If all configured Expressions evaluate to true - for a given FlowFile, that FlowFile will be routed to 'matched'; otherwise, the FlowFile will be - routed to 'unmatched'.
      • - -
      • Route to 'match' if any matches - If any configured Expression evaluates to true - for a given FlowFile, that FlowFile will be routed to 'matched'; otherwise, the FlowFile will be - routed to 'unmatched'.
      • -
      -
    • -
    • Default value: Route to Property Name
    • -
    • Supports expression language: false
    • -
  • -
- -

- Relationships: -

-
    -
  • matched -
      -
    • If the Routing Strategy selected is either Route to 'match' - if all match or Route to 'match' if any matches, then this - relationship becomes available and FlowFiles meeting those - conditions follow this relationship.
    • -
    -
  • -
  • unmatched -
      -
    • FlowFiles that do not match any user-defined expression - will be routed to this relationship.
    • -
    -
  • -
  • user-added relationships -
      -
    • User-added properties result in new relationships; if a - FlowFile's attributes match the criteria specified in the - user-added properties, then it follows the associated - relationship(s).
    • -
    -
  • -
- - - - - diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.RouteOnContent/index.html b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.RouteOnContent/index.html deleted file mode 100644 index 039a142b1d..0000000000 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.RouteOnContent/index.html +++ /dev/null @@ -1,82 +0,0 @@ - - - - - - RouteOnContent - - - - - - -

Description:

-

- This processor applies user-added regular expressions to the content of a FlowFile and routes a copy of the FlowFile to each destination whose regular expression matches. The user adds properties where the name is the relationship that the FlowFile should follow if it matches the regular expression, which is defined as the property's value. User-defined properties do support the NiFi Expression Language, but in such cases, the results are interpreted as literal values, not regular expressions. -

- - Properties: -

-

- In the list below, the names of required properties appear in bold. Any other properties (not in bold) are considered optional. If a property has a default value, it is indicated. If a property supports the use of the NiFi Expression Language (or simply, "expression language"), that is also indicated. -

-
    -
  • Match Requirement -
      -
    • Specifies whether the entire content of the FlowFile must match the regular expression exactly, or if any part of the content (up to a certain point) can contain the regular expression in order to be considered a match. Available options: -
        -
      • content must match exactly
      • -
      • content must contain match
      • -
    • -
    • Default value: content must match exactly
    • -
    • Supports expression language: false
    • -
  • -
  • Character Set -
      -
    • The character set in which the file is encoded.
    • -
    • Default value: UTF-8
    • -
    • Supports expression language: false
    • -
  • -
  • Content Buffer Size -
      -
    • Specifies the maximum amount of data to buffer in order to apply the regular expressions. If the size of the FlowFile exceeds this value, any amount of this value is ignored.
    • -
    • Default value: 1 MB
    • -
    • Supports expression language: false
    • -
  • -
  • User-Defined Properties -
      -
    • Users add properties with regular expressions (see the description above).
    • -
    • User-defined properties do support the NiFi Expression Language, but in such cases, the results are interpreted as literal values, not regular expressions.
    • -
  • -
- -

- Relationships: -

-
    -
  • unmatched -
      -
    • FlowFiles that do not match any user-defined criteria follow this relationship.
    • -
  • -
  • user-defined relationships -
      -
    • User-added properties result in new relationships; if a FlowFile's content matches the criteria specified in the user-added properties, then it follows the associated relationship(s).
    • -
  • -
- - - - diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.ScanAttribute/index.html b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.ScanAttribute/index.html deleted file mode 100644 index 3c4317faa0..0000000000 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.ScanAttribute/index.html +++ /dev/null @@ -1,85 +0,0 @@ - - - - - - ScanAttribute - - - - - - -

Description:

-

- This processor scans the specified attributes of FlowFiles, checking to see if any of their values are present within the specified dictionary of terms. -

- - Properties: -

-

- In the list below, the names of required properties appear in bold. Any other properties (not in bold) are considered optional. If a property has a default value, it is indicated. If a property supports the use of the NiFi Expression Language (or simply, "expression language"), that is also indicated. -

-
    -
  • Dictionary File -
      -
    • The local file path to a new-line-delimited text file that includes terms that should trigger a match. Empty lines are ignored.
    • -
    • Default value: no default
    • -
    • Supports expression language: false
    • -
  • -
  • Attribute Pattern -
      -
    • A Java regular expression that specifies the names of attributes whose values will be matched against the terms in the dictionary file.
    • -
    • Default value: .*
    • -
    • Supports expression language: false
    • -
  • -
  • Match Criteria -
      -
    • If set to "All Must Match", then FlowFiles will be routed to the matched relationship only if all specified attributes' values are found in the dictionary file. If set to "At Least 1 Must Match", then FlowFiles will be routed to "matched" if any attribute specified is found in the dictionary file.
    • -
    • Default value: At Least 1 Must Match
    • -
    • Supports expression language: false
    • -
  • - -
  • Dictionary Filter Pattern -
      -
    • A Java regular expression that will be applied to each line in the dictionary file. If the regular expression does not match the line, then the line will not be included in the list of terms to search for. If a Matching Group is specified, only the portion of the term that matches that Matching Group will be used instead of the entire term. If not specified, all terms in the dictionary will be used and each term will consist of the text of the entire line in the file.
    • -
    • Supports expression language: false
    • -
  • -
- -

- Relationships: -

-
    -
  • matched -
      -
    • FlowFiles whose attributes are found in the dictionary file follow this relationship.
    • -
  • -
  • unmatched -
      -
    • FlowFiles whose attributes are not found in the dictionary file follow this relationship.
    • -
  • -
- - - diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.ScanContent/additionalDetails.html b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.ScanContent/additionalDetails.html new file mode 100644 index 0000000000..5be4580fa5 --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.ScanContent/additionalDetails.html @@ -0,0 +1,45 @@ + + + + + + ScanContent + + + + + + + +

+ Modifies Attributes: +

+ + + + + + + + + + + + + +
Attribute NameDescription
matching.termThe term that caused the Processor to route the FlowFile to the 'matched' relationship; if FlowFile is routed to the 'unmatched' relationship, this attribute is not added.
+ + diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.ScanContent/index.html b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.ScanContent/index.html deleted file mode 100644 index aa49861023..0000000000 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.ScanContent/index.html +++ /dev/null @@ -1,100 +0,0 @@ - - - - - - ScanContent - - - - - - -

Description:

-

This processor scans the content of FlowFiles for terms that - are found in a user-supplied dictionary file. If a term is matched, - the UTF-8 encoded version of the term is added to the FlowFile using - the matching.term attribute. This allows for follow-on processors to - use the value of the matching.term attribute to make routing decisions - and so forth. -

- -

- Modifies Attributes: -

- - - - - - - - - - - - - -
Attribute NameDescription
matching.termThe term that caused the Processor to route the FlowFile to the 'matched' relationship; if FlowFile is routed to the 'unmatched' relationship, this attribute is not added.
- -

- Properties: -

-

In the list below, the names of required properties appear - in bold. Any other properties (not in bold) are considered optional. - If a property has a default value, it is indicated. If a property - supports the use of the NiFi Expression Language (or simply, - "expression language"), that is also indicated.

-
    -
  • Dictionary File -
      -
    • The local file path and filename for the dictionary file.
    • -
    • Default value: no default
    • -
    • Supports expression language: false
    • -
  • -
  • Dictionary Encoding -
      -
    • Indicates how the dictionary file is encoded. If the - default value text is used, the dictionary terms are expected to be - new-line delimited and UTF-8 encoded. If the value binary is used, - the dictionary terms are expected to be denoted by a 4-byte integer - indicating the term length, followed by the term itself.
    • -
    • Default value: text
    • -
    • Supports expression language: false
    • -
  • -
- -

- Relationships: -

-
    -
  • matched -
      -
    • If FlowFiles match at least one term in the dictionary - file, then they follow this relationship.
    • -
    -
  • -
  • unmatched -
      -
    • If FlowFiles do not match any term in the dictionary file, - then they follow this relationship.
    • -
    -
  • -
- - - - diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.SegmentContent/index.html b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.SegmentContent/additionalDetails.html similarity index 68% rename from nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.SegmentContent/index.html rename to nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.SegmentContent/additionalDetails.html index d969fd9443..9e71b83dc6 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.SegmentContent/index.html +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.SegmentContent/additionalDetails.html @@ -23,10 +23,7 @@ -

Description:

-

This processor segments a FlowFile into multiple smaller segments on byte boundaries. Each segment is given attributes that can - then be used by the MergeContent processor to reconstruct the original FlowFile.

- +

Adds or Modifies Attributes:

@@ -74,44 +71,6 @@ - -

- Properties: -

-

In the list below, the names of required properties appear - in bold. Any other properties (not in bold) are considered optional. - If a property has a default value, it is indicated. If a property - supports the use of the NiFi Expression Language (or simply, - "expression language"), that is also indicated.

-
    -
  • Segment Size -
      -
    • The maximum data size for each segment. The value must be a - non-negative integer followed by a supported Data Unit, such as B, - KB, MB, GB, or TB. Example: 1 MB.
    • -
    • Default value: no default
    • -
    • Supports expression language: false
    • -
  • -
- -

- Relationships: -

-
    -
  • original -
      -
    • The original FlowFile will be sent to this relationship.
    • -
    -
  • -
  • segments -
      -
    • All segments will be sent to this relationship. If the file - was small enough that it was not segmented, then a copy of the - original is sent to this relationship as well as original.
    • -
    -
  • -
-

See Also:

diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.SplitContent/index.html b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.SplitContent/additionalDetails.html similarity index 60% rename from nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.SplitContent/index.html rename to nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.SplitContent/additionalDetails.html index 98e726091c..fec022f762 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.SplitContent/index.html +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.SplitContent/additionalDetails.html @@ -23,11 +23,7 @@ -

Description:

-

- This processor splits incoming FlowFiles by a specified byte sequence. -

- +

Modifies Attributes:

@@ -60,42 +56,6 @@ - -

- Properties: -

-

- In the list below, the names of required properties appear in bold. Any other properties (not in bold) are considered optional. If a property has a default value, it is indicated. If a property supports the use of the NiFi Expression Language (or simply, "expression language"), that is also indicated. -

-
    -
  • Byte Sequence -
      -
    • A hex representation of bytes to look for and upon which to split the source file into separate files. This value must be a valid hex string.
    • -
    • Default value: no default
    • -
    • Supports expression language: false
    • -
  • -
  • Keep Byte Sequence -
      -
    • A Boolean value (true/false), indicating whether the byte sequence should be included at the end of each split.
    • -
    • Default value: false
    • -
    • Supports expression language: false
    • -
  • -
- -

- Relationships: -

-
    -
  • original -
      -
    • A copy of the original FlowFile follows this relationship.
    • -
  • -
  • splits -
      -
    • The resulting split files that are produced by this processor follow this relationship.
    • -
  • -
-

See Also:

diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.SplitText/index.html b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.SplitText/additionalDetails.html similarity index 54% rename from nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.SplitText/index.html rename to nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.SplitText/additionalDetails.html index 3e999abece..b5588ac18f 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.SplitText/index.html +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.SplitText/additionalDetails.html @@ -23,11 +23,7 @@ -

Description:

-

- This processor splits a text file into multiple smaller text files on line boundaries, each having up to a configured number of lines. -

- +

Modifies Attributes:

@@ -65,51 +61,6 @@ -

- Properties: -

-

- In the list below, the names of required properties appear in bold. Any other properties (not in bold) are considered optional. If a property has a default value, it is indicated. If a property supports the use of the NiFi Expression Language (or simply, "expression language"), that is also indicated. -

-
    -
  • Line Split Count -
      -
    • The number of lines to be included in each split file. The value must be a non-negative integer.
    • -
    • Default value: no default
    • -
    • Supports expression language: false
    • -
  • -
  • Header Line Count -
      -
    • The number of lines that should be considered part of a header on each split file.
    • -
    • Default value: 0
    • -
    • Supports expression language: false
    • -
  • -
  • Remove Trailing Newlines -
      -
    • Whether to remove newlines at the end of each split file. This should be false if you intend to merge the split files later.
    • -
    • Default value: true
    • -
    • Supports expression language: false
    • -
  • -
- -

- Relationships: -

-
    -
  • failure -
      -
    • If FlowFiles cannot be split for some reason, the original file follows this relationship and does not follow the original relationship.
    • -
  • -
  • original -
      -
    • If FlowFiles are successfully split into one or more files, a copy of the original file follows this relationship.
    • -
  • -
  • splits -
      -
    • If FlowFiles are successfully split into one or more files, those split files follow this relationship.
    • -
  • -
-

See Also:

diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.SplitXml/index.html b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.SplitXml/index.html deleted file mode 100644 index 87700fff76..0000000000 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.SplitXml/index.html +++ /dev/null @@ -1,64 +0,0 @@ - - - - - - SplitXML - - - - - - -

Description:

-

- This processor splits an XML file into multiple separate FlowFiles, each comprising a child or descendant of the original root element. -

- - Properties: -

-

- In the list below, the names of required properties appear in bold. Any other properties (not in bold) are considered optional. If a property has a default value, it is indicated. If a property supports the use of the NiFi Expression Language (or simply, "expression language"), that is also indicated. -

-
    -
  • Line Depth -
      -
    • The XML-nesting depth at which to start splitting the XML fragments. A depth of 1 means to split the root's children; whereas a depth of 2 means to split the root's children's children, and so forth.
    • -
    • Default value: 1
    • -
    • Supports expression language: false
    • -
  • -
- -

- Relationships: -

-
    -
  • failure -
      -
    • If a FlowFile fails processing for any reason (for example, if the FlowFile is not valid XML), then it follows this relationship and does not go down the original relationship.
    • -
  • -
  • original -
      -
    • If FlowFiles are successfully split, a copy of the original FlowFile follows this relationship.
    • -
  • -
  • splits -
      -
    • If FlowFiles are successfully split into one or more files, those split files follow this relationship.
    • -
  • -
- - - diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.TransformXml/index.html b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.TransformXml/index.html deleted file mode 100644 index 2b08d8bef2..0000000000 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.TransformXml/index.html +++ /dev/null @@ -1,63 +0,0 @@ - - - - - - TransformXml - - - - - - -

Description:

-

- This processor transforms the contents of FlowFiles based on a user-specified XSLT stylesheet file. XSL versions 1.0 and 2.0 are supported. -

- - Properties: -

-

- In the list below, the names of required properties appear in bold. Any other properties (not in bold) are considered optional. If a property has a default value, it is indicated. If a property supports the use of the NiFi Expression Language (or simply, "expression language"), that is also indicated. - -

-
    -
  • XSLT File -
      -
    • The local file path to the XLST file to be used for transformation.
    • -
    • Default value: no default
    • -
    • Supports expression language: false
    • -
  • -
- - Note that this processor passes all dynamic properties to the transformer, so therefore they will be available for use in the stylesheet. - -

- Relationships: -

-
    -
  • success -
      -
    • If the FlowFile content is successfully transformed by the XSLT transformer, then the FlowFile follows this relationship.
    • -
  • -
  • failure -
      -
    • If the FlowFiles content fails to be transformed by the XSLT transformer, then the FlowFile follows this relationship.
    • -
  • -
- - - diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.UnpackContent/index.html b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.UnpackContent/additionalDetails.html similarity index 66% rename from nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.UnpackContent/index.html rename to nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.UnpackContent/additionalDetails.html index b3a522e50f..912be354a8 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.UnpackContent/index.html +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.UnpackContent/additionalDetails.html @@ -23,12 +23,7 @@ -

Description:

-

This processor unpacks the content of FlowFiles that have - been packaged with one of several different packaging formats, - emitting one to many FlowFiles for each input FlowFile. -

- +

Uses Attributes:

@@ -99,59 +94,7 @@ - -

- Properties: -

-

In the list below, the names of required properties appear - in bold. Any other properties (not in bold) are considered optional. - If a property has a default value, it is indicated. If a property - supports the use of the NiFi Expression Language (or simply, - "expression language"), that is also indicated.

-
    -
  • Packaging Format -
      -
    • The packaging format used to create incoming FlowFiles. - Valid options include: -
        -
      • use mime.type attribute
      • -
      • tar
      • -
      • zip
      • -
      • flowfile-stream-v3
      • -
      • flowfile-stream-v2
      • -
      • flowfile-tar-v1
      • -
      -
    • -
    • Default value: use mime.type attribute
    • -
    • Supports expression language: false
    • -
  • -
- -

- Relationships: -

-
    -
  • failure -
      -
    • If FlowFiles cannot be unpacked for some reason, the - original copy follows this relationship and does not go down the - original relationship.
    • -
    -
  • -
  • original -
      -
    • If FlowFiles are successfully unpacked, then a copy of each - original incoming FlowFile follows this relationship.
    • -
    -
  • -
  • success -
      -
    • If FlowFiles are successfully unpacked, then the unpacked - FlowFiles that are generated follow this relationship.
    • -
    -
  • -
- +

See Also:

diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-reporting-tasks/src/main/resources/docs/org.apache.nifi.controller.ControllerStatusReportingTask/index.html b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-reporting-tasks/src/main/resources/docs/org.apache.nifi.controller.ControllerStatusReportingTask/additionalDetails.html similarity index 87% rename from nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-reporting-tasks/src/main/resources/docs/org.apache.nifi.controller.ControllerStatusReportingTask/index.html rename to nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-reporting-tasks/src/main/resources/docs/org.apache.nifi.controller.ControllerStatusReportingTask/additionalDetails.html index 656cf40baa..dd7cd5974f 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-reporting-tasks/src/main/resources/docs/org.apache.nifi.controller.ControllerStatusReportingTask/index.html +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-reporting-tasks/src/main/resources/docs/org.apache.nifi.controller.ControllerStatusReportingTask/additionalDetails.html @@ -66,20 +66,5 @@ org.apache.nifi.controller.ControllerStatusReportingTask.Processors and org.apache.nifi.controller.ControllerStatusReportingTask.Connections, respectively.

- -

- Properties: -

- -
    -
  • Show Deltas -
      -
    • Boolean value indicating whether or not the table should display the deltas for values between successive iterations -
    • Default value: true
    • -
    • Supports expression language: false
    • -
    -
  • -
- diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-reporting-tasks/src/main/resources/docs/org.apache.nifi.controller.MonitorDiskUsage/index.html b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-reporting-tasks/src/main/resources/docs/org.apache.nifi.controller.MonitorDiskUsage/index.html deleted file mode 100644 index e02b6493c4..0000000000 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-reporting-tasks/src/main/resources/docs/org.apache.nifi.controller.MonitorDiskUsage/index.html +++ /dev/null @@ -1,58 +0,0 @@ - - - - - - MonitorDiskUsage - - - - -

Description:

- -

Reporting Task that checks how much Disk Space is - available in the Content Repository and the FlowFile Repository. When the amount of available disk space drops below a - specified threshold, this Reporting Task will create a WARNING level log message and create - a System-Level bulletin to notify the user.

- -

- Properties: -

- -
    -
  • Content Repository Threshold -
      -
    • The threshold at which a bulletin will be generated to indicate that the disk usage of the Content Repository is of concern. - The value must be a percentage (e.g., 80%). If multiple directories are configured for the Content Repository, a warning - will be generated for each directory that exceeds the configured limit
    • -
    • Required: true
    • -
    • Default value: 80%
    • -
    • Supports expression language: false
    • -
    -
  • -
  • FlowFile Repository Threshold -
      -
    • The threshold at which a bulletin will be generated to indicate that the disk usage of the FlowFile Repository is of concern. - The value must be a percentage (e.g., 80%).
    • -
    • Required: true
    • -
    • Default value: 80%
    • -
    • Supports expression language: false
    • -
    -
  • -
- - - diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-reporting-tasks/src/main/resources/docs/org.apache.nifi.controller.MonitorMemory/index.html b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-reporting-tasks/src/main/resources/docs/org.apache.nifi.controller.MonitorMemory/index.html deleted file mode 100644 index 08ec0bff16..0000000000 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-reporting-tasks/src/main/resources/docs/org.apache.nifi.controller.MonitorMemory/index.html +++ /dev/null @@ -1,77 +0,0 @@ - - - - - - MonitorMemory - - - - -

Description:

- -

Reporting Task that checks how much Java Heap Space is - available after Full Garbage Collections. When the heap exceeds a - specified threshold immediatley following a Full Garbage Collection, - this Reporting Task will create a WARNING level log message and create - a System-Level bulletin to notify the user.

- -

- Properties: -

- -
    -
  • Memory Pool -
      -
    • The name of the JVM Memory Pool. The allowed values depend - on the JVM and operating system. The following values are typically supported: -
        -
      • PS Old Gen
      • -
      • PS Survivor Space
      • -
      • PS Eden Space
      • -
      • PS Perm Gen
      • -
      -
    • -
    • Required: true
    • -
    • Default value: (none)
    • -
    • Supports expression language: false
    • -
    -
  • -
  • Usage Threshold -
      -
    • - The threshold for memory usage that will cause a notification to occur. The format can either be a percentage (e.g., 80%) or a Data Size (e.g., 1 GB) -
    • -
    • Required: true
    • -
    • Default value: 65%
    • -
    • Supports expression language: false
    • -
    -
  • -
  • Reporting Interval -
      -
    • - Indicates how often this reporting task should report bulletins while the memory utilization exceeds the configured threshold. - If set, must be a valid time period (e.g., "5 mins") -
    • -
    • Required: false
    • -
    • Default value: (none)
    • -
    • Supports expression language: false
    • -
    -
  • -
- - - diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-reporting-tasks/src/main/resources/docs/org.apache.nifi.reporting.ganglia.StandardGangliaReporter/index.html b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-reporting-tasks/src/main/resources/docs/org.apache.nifi.reporting.ganglia.StandardGangliaReporter/additionalDetails.html similarity index 76% rename from nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-reporting-tasks/src/main/resources/docs/org.apache.nifi.reporting.ganglia.StandardGangliaReporter/index.html rename to nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-reporting-tasks/src/main/resources/docs/org.apache.nifi.reporting.ganglia.StandardGangliaReporter/additionalDetails.html index 48df9a6ee2..9818592ba1 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-reporting-tasks/src/main/resources/docs/org.apache.nifi.reporting.ganglia.StandardGangliaReporter/index.html +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-reporting-tasks/src/main/resources/docs/org.apache.nifi.reporting.ganglia.StandardGangliaReporter/additionalDetails.html @@ -37,29 +37,5 @@
  • Bytes Queued: The total number of bytes allocated by the FlowFiles that are currently queued on the system at the point in time at which the Reporting Task is run
  • Active Threads: The number of threads actively running at the point in time at which the Reporting Task is run
  • - -

    - Properties: -

    - -
      -
    • Hostname -
        -
      • The fully-qualified name of the host on which Ganglia is running
      • -
      • Required: true
      • -
      • Default value: localhost
      • -
      • Supports expression language: false
      • -
      -
    • -
    • Port -
        -
      • The Port on which Ganglia is listening for incoming connections
      • -
      • Required: true
      • -
      • Default value: 8649
      • -
      • Supports expression language: false
      • -
      -
    • -
    - diff --git a/nifi/nifi-nar-bundles/nifi-standard-services/nifi-distributed-cache-services-bundle/nifi-distributed-cache-client-service/src/main/resources/docs/org.apache.nifi.distributed.cache.client.DistributedMapCacheClientService/index.html b/nifi/nifi-nar-bundles/nifi-standard-services/nifi-distributed-cache-services-bundle/nifi-distributed-cache-client-service/src/main/resources/docs/org.apache.nifi.distributed.cache.client.DistributedMapCacheClientService/additionalDetails.html similarity index 57% rename from nifi/nifi-nar-bundles/nifi-standard-services/nifi-distributed-cache-services-bundle/nifi-distributed-cache-client-service/src/main/resources/docs/org.apache.nifi.distributed.cache.client.DistributedMapCacheClientService/index.html rename to nifi/nifi-nar-bundles/nifi-standard-services/nifi-distributed-cache-services-bundle/nifi-distributed-cache-client-service/src/main/resources/docs/org.apache.nifi.distributed.cache.client.DistributedMapCacheClientService/additionalDetails.html index 3e5197a5ff..a3bc60f043 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-services/nifi-distributed-cache-services-bundle/nifi-distributed-cache-client-service/src/main/resources/docs/org.apache.nifi.distributed.cache.client.DistributedMapCacheClientService/index.html +++ b/nifi/nifi-nar-bundles/nifi-standard-services/nifi-distributed-cache-services-bundle/nifi-distributed-cache-client-service/src/main/resources/docs/org.apache.nifi.distributed.cache.client.DistributedMapCacheClientService/additionalDetails.html @@ -21,52 +21,6 @@ -

    Description:

    - -

    A Controller Service that can be used to communicate with a - Distributed Map Cache Server.

    - - - -

    - Properties: -

    -

    In the list below, the names of required properties appear - in bold. Any other properties (not in bold) are considered optional. - If a property has a default value, it is indicated. If a property - supports the use of the NiFi Expression Language (or simply, - "expression language"), that is also indicated.

    - -
      -
    • Server Hostname -
        -
      • The name of the server that is running the DistributedMapCacheServer service
      • -
      • Default value: no default
      • -
      • Supports expression language: false
      • -
    • -
    • Server Port -
        -
      • The port on the remote server that is to be used when communicating with the - DistributedMapCacheServer service
      • - -
      • Default value: 4557
      • -
      • Supports expression language: false
      • -
    • -
    • SSL Context Service -
        -
      • If specified, indicates the SSL Context Service that is used to communicate with the remote server. If not specified, communications will not be encrypted -
      • Default value: no default
      • -
      • Supports expression language: false
      • -
    • -
    • Communications Timeout -
        -
      • Specifices how long to wait when communicating with the remote server before determining that there is a communications failure if data cannot be sent or received -
      • Default value: 30 secs
      • -
      • Supports expression language: false
      • -
    • - -
    -

    Below is an example of how to create a client connection to your distributed map cache server. Note that the identifier in this example is cache-client. If you are using this template diff --git a/nifi/nifi-nar-bundles/nifi-standard-services/nifi-distributed-cache-services-bundle/nifi-distributed-cache-client-service/src/main/resources/docs/org.apache.nifi.distributed.cache.client.DistributedSetCacheClientService/index.html b/nifi/nifi-nar-bundles/nifi-standard-services/nifi-distributed-cache-services-bundle/nifi-distributed-cache-client-service/src/main/resources/docs/org.apache.nifi.distributed.cache.client.DistributedSetCacheClientService/additionalDetails.html old mode 100755 new mode 100644 similarity index 70% rename from nifi/nifi-nar-bundles/nifi-standard-services/nifi-distributed-cache-services-bundle/nifi-distributed-cache-client-service/src/main/resources/docs/org.apache.nifi.distributed.cache.client.DistributedSetCacheClientService/index.html rename to nifi/nifi-nar-bundles/nifi-standard-services/nifi-distributed-cache-services-bundle/nifi-distributed-cache-client-service/src/main/resources/docs/org.apache.nifi.distributed.cache.client.DistributedSetCacheClientService/additionalDetails.html index ed1801c893..fa1bf42100 --- a/nifi/nifi-nar-bundles/nifi-standard-services/nifi-distributed-cache-services-bundle/nifi-distributed-cache-client-service/src/main/resources/docs/org.apache.nifi.distributed.cache.client.DistributedSetCacheClientService/index.html +++ b/nifi/nifi-nar-bundles/nifi-standard-services/nifi-distributed-cache-services-bundle/nifi-distributed-cache-client-service/src/main/resources/docs/org.apache.nifi.distributed.cache.client.DistributedSetCacheClientService/additionalDetails.html @@ -21,26 +21,6 @@ -

    Description:

    - -

    A Controller Service that can be used to communicate with a - Distributed Set Cache Server.

    - - - -

    - Properties: -

    -

    In the list below, the names of required properties appear - in bold. Any other properties (not in bold) are considered optional. - If a property has a default value, it is indicated. If a property - supports the use of the NiFi Expression Language (or simply, - "expression language"), that is also indicated.

    - -
      -
    - - See Also:
    • Distributed Map Cache Server
    • diff --git a/nifi/nifi-nar-bundles/nifi-standard-services/nifi-distributed-cache-services-bundle/nifi-distributed-cache-server/src/main/resources/docs/org.apache.nifi.distributed.cache.server.map.DistributedMapCacheServer/index.html b/nifi/nifi-nar-bundles/nifi-standard-services/nifi-distributed-cache-services-bundle/nifi-distributed-cache-server/src/main/resources/docs/org.apache.nifi.distributed.cache.server.map.DistributedMapCacheServer/additionalDetails.html similarity index 54% rename from nifi/nifi-nar-bundles/nifi-standard-services/nifi-distributed-cache-services-bundle/nifi-distributed-cache-server/src/main/resources/docs/org.apache.nifi.distributed.cache.server.map.DistributedMapCacheServer/index.html rename to nifi/nifi-nar-bundles/nifi-standard-services/nifi-distributed-cache-services-bundle/nifi-distributed-cache-server/src/main/resources/docs/org.apache.nifi.distributed.cache.server.map.DistributedMapCacheServer/additionalDetails.html index 918cb22594..2171be246e 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-services/nifi-distributed-cache-services-bundle/nifi-distributed-cache-server/src/main/resources/docs/org.apache.nifi.distributed.cache.server.map.DistributedMapCacheServer/index.html +++ b/nifi/nifi-nar-bundles/nifi-standard-services/nifi-distributed-cache-services-bundle/nifi-distributed-cache-server/src/main/resources/docs/org.apache.nifi.distributed.cache.server.map.DistributedMapCacheServer/additionalDetails.html @@ -21,56 +21,6 @@ -

      Description:

      - -

      A Controller Service that starts an embedded server and listens for connections from clients. The - server provides the ability to query the cache, add data to the cache, and remove data from the cache.

      - - - -

      - Properties: -

      -

      In the list below, the names of required properties appear - in bold. Any other properties (not in bold) are considered optional. - If a property has a default value, it is indicated. If a property - supports the use of the NiFi Expression Language (or simply, - "expression language"), that is also indicated.

      - -
        -
      • Port -
          -
        • The port to listen on for incoming connections
        • -
        • Default value: 4557
        • -
        • Supports expression language: false
        • -
      • -
      • SSL Context Service -
          -
        • If specified, this service will be used to create an SSL Context that will be used to secure communications; if not specified, communications will not be secure
        • -
        • Default value: no default
        • -
        • Supports expression language: false
        • -
      • -
      • Maximum Cache Entries -
          -
        • The maximum number of cache entries that the cache can hold -
        • Default value: 10,000
        • -
        • Supports expression language: false
        • -
      • -
      • Eviction Strategy -
          -
        • Determines which strategy should be used to evict values from the cache to make room for new entries. Valid values: - Least Frequently Used, Least Recently Used, and First In, First Out -
        • Default value: Least Frequently Used
        • -
        • Supports expression language: false
        • -
      • -
      • Persistence Directory -
          -
        • If specified, the cache will be persisted in the given directory; if not specified, the cache will be in-memory only
        • -
        • Default value: no default (in-memory)
        • -
        • Supports expression language: true - JVM and System Properties Only
        • -
      • -
      -

      Below is an example of how to create a distributed map cache server for clients to connect to. Note that the identifier in this example is cache-server. If you are using this template diff --git a/nifi/nifi-nar-bundles/nifi-standard-services/nifi-ssl-context-bundle/nifi-ssl-context-service/src/main/resources/docs/org.apache.nifi.ssl.StandardSSLContextService/index.html b/nifi/nifi-nar-bundles/nifi-standard-services/nifi-ssl-context-bundle/nifi-ssl-context-service/src/main/resources/docs/org.apache.nifi.ssl.StandardSSLContextService/additionalDetails.html similarity index 77% rename from nifi/nifi-nar-bundles/nifi-standard-services/nifi-ssl-context-bundle/nifi-ssl-context-service/src/main/resources/docs/org.apache.nifi.ssl.StandardSSLContextService/index.html rename to nifi/nifi-nar-bundles/nifi-standard-services/nifi-ssl-context-bundle/nifi-ssl-context-service/src/main/resources/docs/org.apache.nifi.ssl.StandardSSLContextService/additionalDetails.html index f1eb2373e0..525337df32 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-services/nifi-ssl-context-bundle/nifi-ssl-context-service/src/main/resources/docs/org.apache.nifi.ssl.StandardSSLContextService/index.html +++ b/nifi/nifi-nar-bundles/nifi-standard-services/nifi-ssl-context-bundle/nifi-ssl-context-service/src/main/resources/docs/org.apache.nifi.ssl.StandardSSLContextService/additionalDetails.html @@ -22,20 +22,6 @@ - - -

      Configuring the SSLContext Controller Service:

      -

      - The SSLContext controller service is a mechanism for providing all the security properties that - allow for secure communications between NiFi extensions and other systems. NiFi extensions include - processors, reporting tasks, and other controller services. -

      - -

      - The controller-services.xml file is located in the NiFi conf - directory. The user may set up any number of controller services within this file. -

      -

      Below is an example of the template for a SSLContext controller service. Note that the identifier in this example is ssl-context. If using this template to create your own SSLContext controller diff --git a/nifi/nifi-nar-bundles/nifi-update-attribute-bundle/nifi-update-attribute-processor/src/main/resources/docs/org.apache.nifi.processors.attributes.UpdateAttribute/index.html b/nifi/nifi-nar-bundles/nifi-update-attribute-bundle/nifi-update-attribute-processor/src/main/resources/docs/org.apache.nifi.processors.attributes.UpdateAttribute/additionalDetails.html similarity index 100% rename from nifi/nifi-nar-bundles/nifi-update-attribute-bundle/nifi-update-attribute-processor/src/main/resources/docs/org.apache.nifi.processors.attributes.UpdateAttribute/index.html rename to nifi/nifi-nar-bundles/nifi-update-attribute-bundle/nifi-update-attribute-processor/src/main/resources/docs/org.apache.nifi.processors.attributes.UpdateAttribute/additionalDetails.html From f269861d9d5b7e1d2a54750840831239a16490d1 Mon Sep 17 00:00:00 2001 From: danbress Date: Mon, 2 Feb 2015 21:01:57 -0500 Subject: [PATCH 04/26] NIFI-280 - removing apache license from generated HTML --- .../html/HtmlDocumentationWriter.java | 24 ++----------------- .../src/main/resources/apache.license | 12 ---------- 2 files changed, 2 insertions(+), 34 deletions(-) delete mode 100644 nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/resources/apache.license diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/html/HtmlDocumentationWriter.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/html/HtmlDocumentationWriter.java index f2ce158a43..40618fef96 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/html/HtmlDocumentationWriter.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/html/HtmlDocumentationWriter.java @@ -25,14 +25,13 @@ import javax.xml.stream.XMLOutputFactory; import javax.xml.stream.XMLStreamException; import javax.xml.stream.XMLStreamWriter; -import org.apache.commons.io.IOUtils; import org.apache.commons.lang3.StringUtils; +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.documentation.Tags; import org.apache.nifi.components.AllowableValue; import org.apache.nifi.components.ConfigurableComponent; import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.documentation.DocumentationWriter; -import org.apache.nifi.annotation.documentation.CapabilityDescription; -import org.apache.nifi.annotation.documentation.Tags; /** * Generates HTML documentation for a ConfigurableComponent. This class is used @@ -48,22 +47,6 @@ public class HtmlDocumentationWriter implements DocumentationWriter { */ public static final String ADDITIONAL_DETAILS_HTML = "additionalDetails.html"; - /** - * The apache licence to apply to the top of the generated html - */ - private static final String apacheLicense; - - static { - String value = null; - try { - value = IOUtils.toString(ClassLoader.getSystemResourceAsStream("apache.license")); - } catch (IOException e) { - e.printStackTrace(); - } - - apacheLicense = value; - } - @Override public void write(final ConfigurableComponent configurableComponent, final OutputStream streamToWriteTo, final boolean includesAdditionalDocumentation) throws IOException { @@ -95,9 +78,6 @@ public class HtmlDocumentationWriter implements DocumentationWriter { */ protected void writeHead(final ConfigurableComponent configurableComponent, final XMLStreamWriter xmlStreamWriter) throws XMLStreamException { - - // write the apache license - xmlStreamWriter.writeComment(apacheLicense); xmlStreamWriter.writeStartElement("head"); xmlStreamWriter.writeStartElement("meta"); xmlStreamWriter.writeAttribute("charset", "utf-8"); diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/resources/apache.license b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/resources/apache.license deleted file mode 100644 index d7b32e8090..0000000000 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/resources/apache.license +++ /dev/null @@ -1,12 +0,0 @@ -Licensed to the Apache Software Foundation (ASF) under one or more - contributor license agreements. See the NOTICE file distributed with - this work for additional information regarding copyright ownership. - The ASF licenses this file to You under the Apache License, Version 2.0 - (the "License"); you may not use this file except in compliance with - the License. You may obtain a copy of the License at - http://www.apache.org/licenses/LICENSE-2.0 - Unless required by applicable law or agreed to in writing, software - distributed under the License is distributed on an "AS IS" BASIS, - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - See the License for the specific language governing permissions and - limitations under the License. \ No newline at end of file From 60070706281e8d8cd8ee2a15312f62a1af9e7e0c Mon Sep 17 00:00:00 2001 From: danbress Date: Tue, 3 Feb 2015 22:37:44 -0500 Subject: [PATCH 05/26] NIFI-280 - putting description as an info bubble --- .../html/HtmlDocumentationWriter.java | 71 +++++++++++++------ .../HtmlProcessorDocumentationWriter.java | 4 +- .../example/FullyDocumentedProcessor.java | 3 +- 3 files changed, 53 insertions(+), 25 deletions(-) diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/html/HtmlDocumentationWriter.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/html/HtmlDocumentationWriter.java index 40618fef96..cd2cee6997 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/html/HtmlDocumentationWriter.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/html/HtmlDocumentationWriter.java @@ -52,8 +52,8 @@ public class HtmlDocumentationWriter implements DocumentationWriter { final boolean includesAdditionalDocumentation) throws IOException { try { - XMLStreamWriter xmlStreamWriter = XMLOutputFactory.newInstance().createXMLStreamWriter(streamToWriteTo, - "UTF-8"); + XMLStreamWriter xmlStreamWriter = XMLOutputFactory.newInstance().createXMLStreamWriter( + streamToWriteTo, "UTF-8"); xmlStreamWriter.writeDTD(""); xmlStreamWriter.writeStartElement("html"); xmlStreamWriter.writeAttribute("lang", "en"); @@ -76,8 +76,8 @@ public class HtmlDocumentationWriter implements DocumentationWriter { * @throws XMLStreamException * thrown if there was a problem writing to the stream */ - protected void writeHead(final ConfigurableComponent configurableComponent, final XMLStreamWriter xmlStreamWriter) - throws XMLStreamException { + protected void writeHead(final ConfigurableComponent configurableComponent, + final XMLStreamWriter xmlStreamWriter) throws XMLStreamException { xmlStreamWriter.writeStartElement("head"); xmlStreamWriter.writeStartElement("meta"); xmlStreamWriter.writeAttribute("charset", "utf-8"); @@ -118,7 +118,8 @@ public class HtmlDocumentationWriter implements DocumentationWriter { * thrown if there was a problem writing to the XML stream */ private final void writeBody(final ConfigurableComponent configurableComponent, - final XMLStreamWriter xmlStreamWriter, final boolean hasAdditionalDetails) throws XMLStreamException { + final XMLStreamWriter xmlStreamWriter, final boolean hasAdditionalDetails) + throws XMLStreamException { xmlStreamWriter.writeStartElement("body"); writeDescription(configurableComponent, xmlStreamWriter, hasAdditionalDetails); writeTags(configurableComponent, xmlStreamWriter); @@ -149,16 +150,18 @@ public class HtmlDocumentationWriter implements DocumentationWriter { * @param xmlStreamWriter * @throws XMLStreamException */ - private void writeTags(final ConfigurableComponent configurableComponent, final XMLStreamWriter xmlStreamWriter) - throws XMLStreamException { + private void writeTags(final ConfigurableComponent configurableComponent, + final XMLStreamWriter xmlStreamWriter) throws XMLStreamException { final Tags tags = configurableComponent.getClass().getAnnotation(Tags.class); + xmlStreamWriter.writeStartElement("h3"); + xmlStreamWriter.writeCharacters("Tags: "); + xmlStreamWriter.writeEndElement(); xmlStreamWriter.writeStartElement("p"); if (tags != null) { final String tagString = StringUtils.join(tags.value(), ", "); - xmlStreamWriter.writeCharacters("Tags: "); xmlStreamWriter.writeCharacters(tagString); } else { - xmlStreamWriter.writeCharacters("No Tags provided."); + xmlStreamWriter.writeCharacters("None."); } xmlStreamWriter.writeEndElement(); @@ -178,7 +181,8 @@ public class HtmlDocumentationWriter implements DocumentationWriter { * thrown if there was a problem writing to the XML stream */ protected void writeDescription(final ConfigurableComponent configurableComponent, - final XMLStreamWriter xmlStreamWriter, final boolean hasAdditionalDetails) throws XMLStreamException { + final XMLStreamWriter xmlStreamWriter, final boolean hasAdditionalDetails) + throws XMLStreamException { writeSimpleElement(xmlStreamWriter, "h2", "Description: "); writeSimpleElement(xmlStreamWriter, "p", getDescription(configurableComponent)); if (hasAdditionalDetails) { @@ -224,9 +228,7 @@ public class HtmlDocumentationWriter implements DocumentationWriter { */ protected void writeProperties(final ConfigurableComponent configurableComponent, final XMLStreamWriter xmlStreamWriter) throws XMLStreamException { - xmlStreamWriter.writeStartElement("p"); - writeSimpleElement(xmlStreamWriter, "strong", "Properties: "); - xmlStreamWriter.writeEndElement(); + writeSimpleElement(xmlStreamWriter, "h3", "Properties: "); xmlStreamWriter.writeStartElement("p"); xmlStreamWriter.writeCharacters("In the list below, the names of required properties appear in "); writeSimpleElement(xmlStreamWriter, "strong", "bold"); @@ -253,11 +255,10 @@ public class HtmlDocumentationWriter implements DocumentationWriter { // write the header row xmlStreamWriter.writeStartElement("tr"); writeSimpleElement(xmlStreamWriter, "th", "Name"); - writeSimpleElement(xmlStreamWriter, "th", "Description"); - writeSimpleElement(xmlStreamWriter, "th", "Default"); - writeSimpleElement(xmlStreamWriter, "th", "Values"); + writeSimpleElement(xmlStreamWriter, "th", "Default Value"); + writeSimpleElement(xmlStreamWriter, "th", "Valid Values"); xmlStreamWriter.writeStartElement("th"); - writeLink(xmlStreamWriter, "EL", "../../html/expression-language-guide.html"); + writeLink(xmlStreamWriter, "Expression Language", "../../html/expression-language-guide.html"); xmlStreamWriter.writeEndElement(); xmlStreamWriter.writeEndElement(); @@ -274,17 +275,18 @@ public class HtmlDocumentationWriter implements DocumentationWriter { if (property.isSensitive()) { writeSensitiveImg(xmlStreamWriter); } + writePropertyDescription(xmlStreamWriter, property.getDescription()); xmlStreamWriter.writeEndElement(); - writeSimpleElement(xmlStreamWriter, "td", property.getDescription()); writeSimpleElement(xmlStreamWriter, "td", property.getDefaultValue()); writeValidValues(xmlStreamWriter, property); - writeSimpleElement(xmlStreamWriter, "td", property.isExpressionLanguageSupported() ? "Yes" : "No"); + writeSimpleElement(xmlStreamWriter, "td", property.isExpressionLanguageSupported() ? "Yes" + : "No"); xmlStreamWriter.writeEndElement(); } // TODO support dynamic properties... xmlStreamWriter.writeEndElement(); - + if (containsSensitiveElement) { writeSensitiveImg(xmlStreamWriter); xmlStreamWriter.writeCharacters(" indicates that a property is a sensitive property"); @@ -295,6 +297,26 @@ public class HtmlDocumentationWriter implements DocumentationWriter { } } + private void writePropertyDescription(XMLStreamWriter xmlStreamWriter, String description) + throws XMLStreamException { + xmlStreamWriter.writeCharacters(" "); + xmlStreamWriter.writeStartElement("img"); + xmlStreamWriter.writeAttribute("src", "../../html/images/iconInfo.png"); + xmlStreamWriter.writeAttribute("alt", description); + xmlStreamWriter.writeEndElement(); + + } + + private void writeValidValueDescription(XMLStreamWriter xmlStreamWriter, String description) + throws XMLStreamException { + xmlStreamWriter.writeCharacters(" "); + xmlStreamWriter.writeStartElement("img"); + xmlStreamWriter.writeAttribute("src", "../../html/images/iconInfo.png"); + xmlStreamWriter.writeAttribute("alt", description); + xmlStreamWriter.writeEndElement(); + + } + private void writeSensitiveImg(final XMLStreamWriter xmlStreamWriter) throws XMLStreamException { xmlStreamWriter.writeCharacters(" "); xmlStreamWriter.writeStartElement("img"); @@ -320,7 +342,14 @@ public class HtmlDocumentationWriter implements DocumentationWriter { if (property.getAllowableValues() != null && property.getAllowableValues().size() > 0) { xmlStreamWriter.writeStartElement("ul"); for (AllowableValue value : property.getAllowableValues()) { - writeSimpleElement(xmlStreamWriter, "li", value.getDisplayName()); + xmlStreamWriter.writeStartElement("li"); + xmlStreamWriter.writeCharacters(value.getDisplayName()); + + if (value.getDescription() != null) { + writeValidValueDescription(xmlStreamWriter, value.getDescription()); + } + xmlStreamWriter.writeEndElement(); + } xmlStreamWriter.writeEndElement(); } diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/html/HtmlProcessorDocumentationWriter.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/html/HtmlProcessorDocumentationWriter.java index 26372359df..4cf6ba850e 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/html/HtmlProcessorDocumentationWriter.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/html/HtmlProcessorDocumentationWriter.java @@ -35,9 +35,7 @@ public class HtmlProcessorDocumentationWriter extends HtmlDocumentationWriter { protected void writeAdditionalBodyInfo(final ConfigurableComponent configurableComponent, final XMLStreamWriter xmlStreamWriter) throws XMLStreamException { final Processor processor = (Processor) configurableComponent; - xmlStreamWriter.writeStartElement("p"); - writeSimpleElement(xmlStreamWriter, "strong", "Relationships: "); - xmlStreamWriter.writeEndElement(); + writeSimpleElement(xmlStreamWriter, "h3", "Relationships: "); if (processor.getRelationships().size() > 0) { xmlStreamWriter.writeStartElement("table"); diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/java/org/apache/nifi/documentation/example/FullyDocumentedProcessor.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/java/org/apache/nifi/documentation/example/FullyDocumentedProcessor.java index 2a001779ad..a30133ebb4 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/java/org/apache/nifi/documentation/example/FullyDocumentedProcessor.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/java/org/apache/nifi/documentation/example/FullyDocumentedProcessor.java @@ -22,6 +22,7 @@ import java.util.HashSet; import java.util.List; import java.util.Set; +import org.apache.nifi.components.AllowableValue; import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.processor.AbstractProcessor; import org.apache.nifi.processor.ProcessContext; @@ -44,7 +45,7 @@ public class FullyDocumentedProcessor extends AbstractProcessor { public static final PropertyDescriptor RECURSE = new PropertyDescriptor.Builder().name("Recurse Subdirectories") .description("Indicates whether or not to pull files from subdirectories").required(true) - .allowableValues("true", "false").defaultValue("true").build(); + .allowableValues(new AllowableValue("true", "true", "Should pull from sub directories"), new AllowableValue("false", "false", "Should not pull from sub directories")).defaultValue("true").build(); public static final PropertyDescriptor POLLING_INTERVAL = new PropertyDescriptor.Builder().name("Polling Interval") .description("Indicates how long to wait before performing a directory listing").required(true) From 5f1fb45c91911fffd58e5ab8b4403265236d9b27 Mon Sep 17 00:00:00 2001 From: danbress Date: Tue, 3 Feb 2015 22:51:17 -0500 Subject: [PATCH 06/26] NIFI-280 writing the description as a second row --- .../documentation/html/HtmlDocumentationWriter.java | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/html/HtmlDocumentationWriter.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/html/HtmlDocumentationWriter.java index cd2cee6997..ca815e6da1 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/html/HtmlDocumentationWriter.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/html/HtmlDocumentationWriter.java @@ -275,13 +275,21 @@ public class HtmlDocumentationWriter implements DocumentationWriter { if (property.isSensitive()) { writeSensitiveImg(xmlStreamWriter); } - writePropertyDescription(xmlStreamWriter, property.getDescription()); + //writePropertyDescription(xmlStreamWriter, property.getDescription()); xmlStreamWriter.writeEndElement(); writeSimpleElement(xmlStreamWriter, "td", property.getDefaultValue()); writeValidValues(xmlStreamWriter, property); writeSimpleElement(xmlStreamWriter, "td", property.isExpressionLanguageSupported() ? "Yes" : "No"); xmlStreamWriter.writeEndElement(); + xmlStreamWriter.writeStartElement("tr"); + xmlStreamWriter.writeStartElement("td"); + xmlStreamWriter.writeAttribute("span", "4"); + xmlStreamWriter.writeCharacters("Description:"); + + xmlStreamWriter.writeCharacters(property.getDescription()); + xmlStreamWriter.writeEndElement(); + xmlStreamWriter.writeEndElement(); } // TODO support dynamic properties... From c3085b8b21501c48c8debf504a6ce893af552dc8 Mon Sep 17 00:00:00 2001 From: danbress Date: Tue, 3 Feb 2015 23:18:43 -0500 Subject: [PATCH 07/26] NIFI-280 - fixing colspan --- .../nifi/documentation/html/HtmlDocumentationWriter.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/html/HtmlDocumentationWriter.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/html/HtmlDocumentationWriter.java index ca815e6da1..78d6ac81d5 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/html/HtmlDocumentationWriter.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/html/HtmlDocumentationWriter.java @@ -284,7 +284,8 @@ public class HtmlDocumentationWriter implements DocumentationWriter { xmlStreamWriter.writeEndElement(); xmlStreamWriter.writeStartElement("tr"); xmlStreamWriter.writeStartElement("td"); - xmlStreamWriter.writeAttribute("span", "4"); + xmlStreamWriter.writeAttribute("colspan", "4"); + xmlStreamWriter.writeAttribute("class", "description-row"); xmlStreamWriter.writeCharacters("Description:"); xmlStreamWriter.writeCharacters(property.getDescription()); From 6e80ae7b9bc95c86bc907823be4eb0dd41a369e2 Mon Sep 17 00:00:00 2001 From: danbress Date: Thu, 5 Feb 2015 19:52:04 -0500 Subject: [PATCH 08/26] NIFI-280 - changing the way properties are formatted --- .../html/HtmlDocumentationWriter.java | 76 ++++++++----------- 1 file changed, 30 insertions(+), 46 deletions(-) diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/html/HtmlDocumentationWriter.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/html/HtmlDocumentationWriter.java index 78d6ac81d5..c192759b12 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/html/HtmlDocumentationWriter.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/html/HtmlDocumentationWriter.java @@ -31,6 +31,7 @@ import org.apache.nifi.annotation.documentation.Tags; import org.apache.nifi.components.AllowableValue; import org.apache.nifi.components.ConfigurableComponent; import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.controller.ControllerService; import org.apache.nifi.documentation.DocumentationWriter; /** @@ -232,10 +233,10 @@ public class HtmlDocumentationWriter implements DocumentationWriter { xmlStreamWriter.writeStartElement("p"); xmlStreamWriter.writeCharacters("In the list below, the names of required properties appear in "); writeSimpleElement(xmlStreamWriter, "strong", "bold"); - xmlStreamWriter.writeCharacters(". Any" + xmlStreamWriter.writeCharacters(". Any " + "other properties (not in bold) are considered optional. The table also " + "indicates any default values, whether a property supports the "); - writeLink(xmlStreamWriter, "NiFi Expression Language (or simply EL)", + writeLink(xmlStreamWriter, "NiFi Expression Language", "../../html/expression-language-guide.html"); xmlStreamWriter.writeCharacters(", and whether a property is considered " + "\"sensitive\", meaning that its value will be encrypted. Before entering a " @@ -246,7 +247,6 @@ public class HtmlDocumentationWriter implements DocumentationWriter { xmlStreamWriter.writeCharacters("."); xmlStreamWriter.writeEndElement(); - boolean containsSensitiveElement = false; List properties = configurableComponent.getPropertyDescriptors(); if (properties.size() > 0) { @@ -257,14 +257,11 @@ public class HtmlDocumentationWriter implements DocumentationWriter { writeSimpleElement(xmlStreamWriter, "th", "Name"); writeSimpleElement(xmlStreamWriter, "th", "Default Value"); writeSimpleElement(xmlStreamWriter, "th", "Valid Values"); - xmlStreamWriter.writeStartElement("th"); - writeLink(xmlStreamWriter, "Expression Language", "../../html/expression-language-guide.html"); - xmlStreamWriter.writeEndElement(); + writeSimpleElement(xmlStreamWriter, "th", "Description"); xmlStreamWriter.writeEndElement(); // write the individual properties for (PropertyDescriptor property : properties) { - containsSensitiveElement |= property.isSensitive(); xmlStreamWriter.writeStartElement("tr"); xmlStreamWriter.writeStartElement("td"); if (property.isRequired()) { @@ -272,68 +269,52 @@ public class HtmlDocumentationWriter implements DocumentationWriter { } else { xmlStreamWriter.writeCharacters(property.getDisplayName()); } - if (property.isSensitive()) { - writeSensitiveImg(xmlStreamWriter); - } - //writePropertyDescription(xmlStreamWriter, property.getDescription()); + xmlStreamWriter.writeEndElement(); writeSimpleElement(xmlStreamWriter, "td", property.getDefaultValue()); - writeValidValues(xmlStreamWriter, property); - writeSimpleElement(xmlStreamWriter, "td", property.isExpressionLanguageSupported() ? "Yes" - : "No"); - xmlStreamWriter.writeEndElement(); - xmlStreamWriter.writeStartElement("tr"); xmlStreamWriter.writeStartElement("td"); - xmlStreamWriter.writeAttribute("colspan", "4"); - xmlStreamWriter.writeAttribute("class", "description-row"); - xmlStreamWriter.writeCharacters("Description:"); - - xmlStreamWriter.writeCharacters(property.getDescription()); + writeValidValues(xmlStreamWriter, property); xmlStreamWriter.writeEndElement(); + xmlStreamWriter.writeStartElement("td"); + if (property.getDescription() != null && property.getDescription().trim().length() > 0) { + xmlStreamWriter.writeCharacters(property.getDescription()); + } else { + xmlStreamWriter.writeCharacters("No Description Provided."); + } + + if (property.isSensitive()) { + xmlStreamWriter.writeEmptyElement("br"); + writeSimpleElement(xmlStreamWriter, "strong", "Sensitive Property: true"); + } + + if (property.isExpressionLanguageSupported()) { + xmlStreamWriter.writeEmptyElement("br"); + writeSimpleElement(xmlStreamWriter, "strong", "Supports Expression Language: true"); + } + xmlStreamWriter.writeEndElement(); + xmlStreamWriter.writeEndElement(); } // TODO support dynamic properties... xmlStreamWriter.writeEndElement(); - if (containsSensitiveElement) { - writeSensitiveImg(xmlStreamWriter); - xmlStreamWriter.writeCharacters(" indicates that a property is a sensitive property"); - } - } else { writeSimpleElement(xmlStreamWriter, "p", "This component has no required or optional properties."); } } - private void writePropertyDescription(XMLStreamWriter xmlStreamWriter, String description) - throws XMLStreamException { - xmlStreamWriter.writeCharacters(" "); - xmlStreamWriter.writeStartElement("img"); - xmlStreamWriter.writeAttribute("src", "../../html/images/iconInfo.png"); - xmlStreamWriter.writeAttribute("alt", description); - xmlStreamWriter.writeEndElement(); - - } - private void writeValidValueDescription(XMLStreamWriter xmlStreamWriter, String description) throws XMLStreamException { xmlStreamWriter.writeCharacters(" "); xmlStreamWriter.writeStartElement("img"); xmlStreamWriter.writeAttribute("src", "../../html/images/iconInfo.png"); xmlStreamWriter.writeAttribute("alt", description); + xmlStreamWriter.writeAttribute("title", description); xmlStreamWriter.writeEndElement(); } - private void writeSensitiveImg(final XMLStreamWriter xmlStreamWriter) throws XMLStreamException { - xmlStreamWriter.writeCharacters(" "); - xmlStreamWriter.writeStartElement("img"); - xmlStreamWriter.writeAttribute("src", "../../html/images/iconSecure.png"); - xmlStreamWriter.writeAttribute("alt", "Sensitive Property"); - xmlStreamWriter.writeEndElement(); - } - /** * Interrogates a PropertyDescriptor to get a list of AllowableValues, if * there are none, nothing is written to the stream. @@ -347,7 +328,6 @@ public class HtmlDocumentationWriter implements DocumentationWriter { */ protected void writeValidValues(XMLStreamWriter xmlStreamWriter, PropertyDescriptor property) throws XMLStreamException { - xmlStreamWriter.writeStartElement("td"); if (property.getAllowableValues() != null && property.getAllowableValues().size() > 0) { xmlStreamWriter.writeStartElement("ul"); for (AllowableValue value : property.getAllowableValues()) { @@ -361,8 +341,12 @@ public class HtmlDocumentationWriter implements DocumentationWriter { } xmlStreamWriter.writeEndElement(); + } else if (property.getControllerServiceDefinition() != null) { + Class controllerServiceClass = property.getControllerServiceDefinition(); + writeSimpleElement(xmlStreamWriter, "strong", "Controller Service: "); + xmlStreamWriter.writeEmptyElement("br"); + xmlStreamWriter.writeCharacters(controllerServiceClass.getSimpleName()); } - xmlStreamWriter.writeEndElement(); } /** From 4ab278dd197c81e40b73f70e03fbb71997be6823 Mon Sep 17 00:00:00 2001 From: danbress Date: Sat, 7 Feb 2015 11:51:04 -0500 Subject: [PATCH 09/26] NIFI-280 fixing up table formatting of tables in components --- .../src/main/webapp/css/component-usage.css | 94 ++++++++++++++----- 1 file changed, 71 insertions(+), 23 deletions(-) diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-docs/src/main/webapp/css/component-usage.css b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-docs/src/main/webapp/css/component-usage.css index 816403427e..f43db46a7f 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-docs/src/main/webapp/css/component-usage.css +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-docs/src/main/webapp/css/component-usage.css @@ -42,30 +42,75 @@ body { /* tables */ table { - background-color: #fefefe; - border: 1px solid #ccc; - border-left: 6px solid #ccc; - color: #555; - display: block; - margin-bottom: 12px; - padding: 5px 8px; + color:#666; + font-size:14px; + background:#eaebec; + border:#ccc 1px solid; + -webkit-border-radius:3px; + border-radius:3px; + width: 100%; } -tr td { - font-size: 14px; - vertical-align:top; - text-align:left; - padding: 4px; - border-width: 0; +table th { + padding:21px 25px 22px 25px; + border-top:1px solid #fafafa; + border-bottom:1px solid #e0e0e0; + + background: #ededed; } -tr th { - font-size: 16px; - vertical-align:top; - text-align:left; - padding: 4px; - border-width: 0; - white-space: nowrap; +table th:first-child { + text-align: left; + padding-left:20px; +} + +table th:last-child { + text-align: left; + padding-left:20px; +} + +table tr:first-child th:first-child { + border-top-left-radius:3px; +} + +table tr:first-child th:last-child { + border-top-right-radius:3px; +} + +table tr { + text-align: center; + padding-left:20px; +} + +table td:first-child { + text-align: left; + padding-left:20px; + border-left: 0; +} + +table td:last-child { + text-align: left; + padding-left:20px; + border-left: 0; + vertical-align: top; + +} + +table td { + padding:18px; + background: #fafafa; +} + +table tr:last-child td { + border-bottom:0; +} + +table tr:last-child td:first-child { + border-bottom-left-radius:3px; +} + +table tr:last-child td:last-child { + border-bottom-right-radius:3px; } /* links */ @@ -99,10 +144,13 @@ p strong { } /* ul li */ - +td ul { + margin: 0px 0px 0px 0px; + padding-left: 20px; +} ul li { - font-family: 'Noto Serif', 'DejaVu Serif', serif; - font-size: 16px; + text-align: left; + display: list-item; } ul li strong { From 9552b189c9861d4eb54bb7a2a569e1447c13332a Mon Sep 17 00:00:00 2001 From: danbress Date: Sat, 7 Feb 2015 11:51:18 -0500 Subject: [PATCH 10/26] NIFI-280 - creating some example to test formatting --- .../nifi/documentation/example/FullyDocumentedProcessor.java | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/java/org/apache/nifi/documentation/example/FullyDocumentedProcessor.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/java/org/apache/nifi/documentation/example/FullyDocumentedProcessor.java index a30133ebb4..e1b8634ff6 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/java/org/apache/nifi/documentation/example/FullyDocumentedProcessor.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/java/org/apache/nifi/documentation/example/FullyDocumentedProcessor.java @@ -53,6 +53,9 @@ public class FullyDocumentedProcessor extends AbstractProcessor { public static final PropertyDescriptor OPTIONAL_PROPERTY = new PropertyDescriptor.Builder() .name("Optional Property").description("This is a property you can use or not").required(false).build(); + + public static final PropertyDescriptor TYPE_PROPERTY = new PropertyDescriptor.Builder() + .name("Type").description("This is the type of something that you can choose. It has several possible values").allowableValues("yes", "no", "maybe", "possibly", "not likely", "longer option name").required(true).build(); public static final Relationship REL_SUCCESS = new Relationship.Builder().name("success") .description("Successful files").build(); @@ -69,6 +72,7 @@ public class FullyDocumentedProcessor extends AbstractProcessor { properties.add(RECURSE); properties.add(POLLING_INTERVAL); properties.add(OPTIONAL_PROPERTY); + properties.add(TYPE_PROPERTY); this.properties = Collections.unmodifiableList(properties); final Set relationships = new HashSet<>(); From ef6d76cbd92b96c8eb9c5d45f0189888c63d8fb9 Mon Sep 17 00:00:00 2001 From: danbress Date: Sat, 7 Feb 2015 11:54:01 -0500 Subject: [PATCH 11/26] NIFI-280 - reducing table padding --- .../src/main/webapp/css/component-usage.css | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-docs/src/main/webapp/css/component-usage.css b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-docs/src/main/webapp/css/component-usage.css index f43db46a7f..45c1e267ac 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-docs/src/main/webapp/css/component-usage.css +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-docs/src/main/webapp/css/component-usage.css @@ -52,7 +52,7 @@ table { } table th { - padding:21px 25px 22px 25px; + padding:11px 15px 12px 15px; border-top:1px solid #fafafa; border-bottom:1px solid #e0e0e0; @@ -61,12 +61,12 @@ table th { table th:first-child { text-align: left; - padding-left:20px; + padding-left:10px; } table th:last-child { text-align: left; - padding-left:20px; + padding-left:10px; } table tr:first-child th:first-child { @@ -79,25 +79,25 @@ table tr:first-child th:last-child { table tr { text-align: center; - padding-left:20px; + padding-left:10px; } table td:first-child { text-align: left; - padding-left:20px; + padding-left:10px; border-left: 0; } table td:last-child { text-align: left; - padding-left:20px; + padding-left:10px; border-left: 0; vertical-align: top; } table td { - padding:18px; + padding:12px; background: #fafafa; } From d53abf02d1fb516cf88cba4a3e30993a8d3bb885 Mon Sep 17 00:00:00 2001 From: danbress Date: Sun, 8 Feb 2015 10:15:51 -0500 Subject: [PATCH 12/26] NIFI-280 - fixing formatting of additionalDetails tables --- .../additionalDetails.html | 2 +- .../additionalDetails.html | 2 +- .../additionalDetails.html | 2 +- .../additionalDetails.html | 4 ++-- .../additionalDetails.html | 2 +- .../additionalDetails.html | 2 +- .../additionalDetails.html | 2 +- .../additionalDetails.html | 2 +- .../additionalDetails.html | 2 +- .../additionalDetails.html | 2 +- .../additionalDetails.html | 2 +- .../additionalDetails.html | 2 +- .../additionalDetails.html | 4 ++-- .../additionalDetails.html | 2 +- .../additionalDetails.html | 2 +- .../additionalDetails.html | 2 +- .../additionalDetails.html | 2 +- .../additionalDetails.html | 2 +- .../additionalDetails.html | 2 +- .../additionalDetails.html | 4 ++-- 20 files changed, 23 insertions(+), 23 deletions(-) diff --git a/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/resources/docs/org.apache.nifi.processors.hadoop.GetHDFS/additionalDetails.html b/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/resources/docs/org.apache.nifi.processors.hadoop.GetHDFS/additionalDetails.html index 0060c2c1f6..cd8cc82d75 100644 --- a/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/resources/docs/org.apache.nifi.processors.hadoop.GetHDFS/additionalDetails.html +++ b/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/resources/docs/org.apache.nifi.processors.hadoop.GetHDFS/additionalDetails.html @@ -25,7 +25,7 @@

      Modifies Attributes:

      - +
      diff --git a/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/resources/docs/org.apache.nifi.processors.hadoop.PutHDFS/additionalDetails.html b/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/resources/docs/org.apache.nifi.processors.hadoop.PutHDFS/additionalDetails.html index 1a02d1085c..17b286b3dd 100644 --- a/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/resources/docs/org.apache.nifi.processors.hadoop.PutHDFS/additionalDetails.html +++ b/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/resources/docs/org.apache.nifi.processors.hadoop.PutHDFS/additionalDetails.html @@ -26,7 +26,7 @@

      Uses Attributes:

      -
      Attribute Name
      +
      diff --git a/nifi/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/resources/docs/org.apache.nifi.processors.kafka.GetKafka/additionalDetails.html b/nifi/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/resources/docs/org.apache.nifi.processors.kafka.GetKafka/additionalDetails.html index daee6ee711..b182189c97 100644 --- a/nifi/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/resources/docs/org.apache.nifi.processors.kafka.GetKafka/additionalDetails.html +++ b/nifi/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/resources/docs/org.apache.nifi.processors.kafka.GetKafka/additionalDetails.html @@ -45,7 +45,7 @@

      Modifies Attributes:

      -
      Attribute Name
      +
      diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.CompressContent/additionalDetails.html b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.CompressContent/additionalDetails.html index 4339e32c05..a85456df7e 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.CompressContent/additionalDetails.html +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.CompressContent/additionalDetails.html @@ -28,7 +28,7 @@

      Uses Attributes:

      -
      Attribute Name
      +
      @@ -47,7 +47,7 @@

      Modifies Attributes:

      -
      Attribute Name
      +
      diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.DetectDuplicate/additionalDetails.html b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.DetectDuplicate/additionalDetails.html index c0b65b9167..188b137af9 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.DetectDuplicate/additionalDetails.html +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.DetectDuplicate/additionalDetails.html @@ -26,7 +26,7 @@

      Modifies Attributes:

      -
      Attribute Name
      +
      diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.ExecuteStreamCommand/additionalDetails.html b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.ExecuteStreamCommand/additionalDetails.html index 0aa703b673..2b1450a01c 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.ExecuteStreamCommand/additionalDetails.html +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.ExecuteStreamCommand/additionalDetails.html @@ -27,7 +27,7 @@

      Creates Attributes:

      -
      Attribute Name
      +
      diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.GetFTP/additionalDetails.html b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.GetFTP/additionalDetails.html index 92d69fd1c0..4805ed8936 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.GetFTP/additionalDetails.html +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.GetFTP/additionalDetails.html @@ -26,7 +26,7 @@

      Modifies Attributes:

      -
      Attribute Name
      +
      diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.GetFile/additionalDetails.html b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.GetFile/additionalDetails.html index 76046c6485..6a3b77f012 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.GetFile/additionalDetails.html +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.GetFile/additionalDetails.html @@ -26,7 +26,7 @@

      Modifies Attributes:

      -
      Attribute Name
      +
      diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.GetHTTP/additionalDetails.html b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.GetHTTP/additionalDetails.html index 94f639bf67..c975303463 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.GetHTTP/additionalDetails.html +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.GetHTTP/additionalDetails.html @@ -27,7 +27,7 @@

      Modifies Attributes:

      -
      Attribute Name
      +
      diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.GetSFTP/additionalDetails.html b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.GetSFTP/additionalDetails.html index 8cb65261c9..74a7cb0d14 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.GetSFTP/additionalDetails.html +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.GetSFTP/additionalDetails.html @@ -27,7 +27,7 @@

      Modifies Attributes:

      -
      Attribute Name
      +
      diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.IdentifyMimeType/additionalDetails.html b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.IdentifyMimeType/additionalDetails.html index 1fb55667f7..22ebd1db03 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.IdentifyMimeType/additionalDetails.html +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.IdentifyMimeType/additionalDetails.html @@ -79,7 +79,7 @@

      Modifies Attributes:

      -
      Attribute Name
      +
      diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.InvokeHTTP/additionalDetails.html b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.InvokeHTTP/additionalDetails.html index f7f06a047b..1c33360d26 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.InvokeHTTP/additionalDetails.html +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.InvokeHTTP/additionalDetails.html @@ -26,7 +26,7 @@ Adds Attributes:

      -
      Attribute Name
      +
      diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.MergeContent/additionalDetails.html b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.MergeContent/additionalDetails.html index 68f70076be..22007648c1 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.MergeContent/additionalDetails.html +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.MergeContent/additionalDetails.html @@ -27,7 +27,7 @@

      Uses Attributes:

      -
      Attribute Name
      +
      @@ -79,7 +79,7 @@

      Modifies Attributes:

      -
      Attribute Name
      +
      diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.MonitorActivity/additionalDetails.html b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.MonitorActivity/additionalDetails.html index e05db70d4f..0a3b66537e 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.MonitorActivity/additionalDetails.html +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.MonitorActivity/additionalDetails.html @@ -27,7 +27,7 @@

      Modifies Attributes:

      -
      Attribute Name
      +
      diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.PostHTTP/additionalDetails.html b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.PostHTTP/additionalDetails.html index 5ba60fac82..f8b15f91c5 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.PostHTTP/additionalDetails.html +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.PostHTTP/additionalDetails.html @@ -29,7 +29,7 @@

      Uses Attributes:

      -
      Attribute Name
      +
      diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.ScanContent/additionalDetails.html b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.ScanContent/additionalDetails.html index 5be4580fa5..f8560f2d0f 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.ScanContent/additionalDetails.html +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.ScanContent/additionalDetails.html @@ -27,7 +27,7 @@

      Modifies Attributes:

      -
      Attribute Name
      +
      diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.SegmentContent/additionalDetails.html b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.SegmentContent/additionalDetails.html index 9e71b83dc6..64dd7baec3 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.SegmentContent/additionalDetails.html +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.SegmentContent/additionalDetails.html @@ -27,7 +27,7 @@

      Adds or Modifies Attributes:

      -
      Attribute Name
      +
      diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.SplitContent/additionalDetails.html b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.SplitContent/additionalDetails.html index fec022f762..64d8d7dce3 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.SplitContent/additionalDetails.html +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.SplitContent/additionalDetails.html @@ -27,7 +27,7 @@

      Modifies Attributes:

      -
      Attribute Name
      +
      diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.SplitText/additionalDetails.html b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.SplitText/additionalDetails.html index b5588ac18f..ad01242dc0 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.SplitText/additionalDetails.html +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.SplitText/additionalDetails.html @@ -27,7 +27,7 @@

      Modifies Attributes:

      -
      Attribute Name
      +
      diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.UnpackContent/additionalDetails.html b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.UnpackContent/additionalDetails.html index 912be354a8..30ac301fad 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.UnpackContent/additionalDetails.html +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.UnpackContent/additionalDetails.html @@ -27,7 +27,7 @@

      Uses Attributes:

      -
      Attribute Name
      +
      @@ -55,7 +55,7 @@

      Modifies Attributes:

      -
      Attribute Name
      +
      From 74857166f8eb757754283960867ad7fda9541d05 Mon Sep 17 00:00:00 2001 From: danbress Date: Sun, 8 Feb 2015 14:34:56 -0500 Subject: [PATCH 13/26] NIFI-280 - adding additional unit tests --- .../example/FullyDocumentedProcessor.java | 5 +++ .../documentation/example/NakedProcessor.java | 16 +++++++ .../documentation/example/SampleService.java | 25 +++++++++++ .../html/HtmlDocumentationWriterTest.java | 43 +++++++++++++++++-- .../ProcessorDocumentationWriterTest.java | 38 +++++++++++++++- 5 files changed, 122 insertions(+), 5 deletions(-) create mode 100644 nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/java/org/apache/nifi/documentation/example/NakedProcessor.java create mode 100644 nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/java/org/apache/nifi/documentation/example/SampleService.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/java/org/apache/nifi/documentation/example/FullyDocumentedProcessor.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/java/org/apache/nifi/documentation/example/FullyDocumentedProcessor.java index e1b8634ff6..c2c765784f 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/java/org/apache/nifi/documentation/example/FullyDocumentedProcessor.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/java/org/apache/nifi/documentation/example/FullyDocumentedProcessor.java @@ -57,6 +57,10 @@ public class FullyDocumentedProcessor extends AbstractProcessor { public static final PropertyDescriptor TYPE_PROPERTY = new PropertyDescriptor.Builder() .name("Type").description("This is the type of something that you can choose. It has several possible values").allowableValues("yes", "no", "maybe", "possibly", "not likely", "longer option name").required(true).build(); + public static final PropertyDescriptor SERVICE_PROPERTY = new PropertyDescriptor.Builder() + .name("Controller Service").description("This is the controller service to use to do things") + .identifiesControllerService(SampleService.class).required(true).build(); + public static final Relationship REL_SUCCESS = new Relationship.Builder().name("success") .description("Successful files").build(); public static final Relationship REL_FAILURE = new Relationship.Builder().name("failure") @@ -73,6 +77,7 @@ public class FullyDocumentedProcessor extends AbstractProcessor { properties.add(POLLING_INTERVAL); properties.add(OPTIONAL_PROPERTY); properties.add(TYPE_PROPERTY); + properties.add(SERVICE_PROPERTY); this.properties = Collections.unmodifiableList(properties); final Set relationships = new HashSet<>(); diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/java/org/apache/nifi/documentation/example/NakedProcessor.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/java/org/apache/nifi/documentation/example/NakedProcessor.java new file mode 100644 index 0000000000..6fce1e1386 --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/java/org/apache/nifi/documentation/example/NakedProcessor.java @@ -0,0 +1,16 @@ +package org.apache.nifi.documentation.example; + +import org.apache.nifi.processor.AbstractProcessor; +import org.apache.nifi.processor.ProcessContext; +import org.apache.nifi.processor.ProcessSession; +import org.apache.nifi.processor.exception.ProcessException; + +public class NakedProcessor extends AbstractProcessor { + + @Override + public void onTrigger(ProcessContext arg0, ProcessSession arg1) throws ProcessException { + // TODO Auto-generated method stub + + } + +} diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/java/org/apache/nifi/documentation/example/SampleService.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/java/org/apache/nifi/documentation/example/SampleService.java new file mode 100644 index 0000000000..6224364f7b --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/java/org/apache/nifi/documentation/example/SampleService.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.example; + +import org.apache.nifi.controller.ControllerService; + +public interface SampleService extends ControllerService { + + public void doSomething(); + +} diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/java/org/apache/nifi/documentation/html/HtmlDocumentationWriterTest.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/java/org/apache/nifi/documentation/html/HtmlDocumentationWriterTest.java index f685f39b74..9d7926e858 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/java/org/apache/nifi/documentation/html/HtmlDocumentationWriterTest.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/java/org/apache/nifi/documentation/html/HtmlDocumentationWriterTest.java @@ -16,6 +16,7 @@ */ package org.apache.nifi.documentation.html; +import java.io.ByteArrayOutputStream; import java.io.IOException; import org.apache.nifi.controller.ControllerService; @@ -28,6 +29,8 @@ import org.apache.nifi.reporting.InitializationException; import org.apache.nifi.reporting.ReportingTask; import org.junit.Test; +import static org.apache.nifi.documentation.html.XmlValidator.assertContains; + public class HtmlDocumentationWriterTest { @Test @@ -38,8 +41,26 @@ public class HtmlDocumentationWriterTest { DocumentationWriter writer = new HtmlDocumentationWriter(); - writer.write(controllerService, System.out, false); + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + writer.write(controllerService, baos, false); + + String results = new String(baos.toByteArray()); + XmlValidator.assertXmlValid(results); + + // description + assertContains(results, "A documented controller service that can help you do things"); + + // tags + assertContains(results, "one, two, three"); + + // properties + assertContains(results, "Keystore Filename"); + assertContains(results, "The fully-qualified filename of the Keystore"); + assertContains(results, "Keystore Type"); + assertContains(results, "JKS"); + assertContains(results, "PKCS12"); + assertContains(results, "Sensitive Property: true"); } @Test @@ -50,7 +71,23 @@ public class HtmlDocumentationWriterTest { DocumentationWriter writer = new HtmlDocumentationWriter(); - writer.write(reportingTask, System.out, false); - } + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + writer.write(reportingTask, baos, false); + + String results = new String(baos.toByteArray()); + XmlValidator.assertXmlValid(results); + + // description + assertContains(results, "A helper reporting task to do..."); + + // tags + assertContains(results, "first, second, third"); + + // properties + assertContains(results, "Show Deltas"); + assertContains(results, "Specifies whether or not to show the difference in values between the current status and the previous status"); + assertContains(results, "true"); + assertContains(results, "false"); + } } diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/java/org/apache/nifi/documentation/html/ProcessorDocumentationWriterTest.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/java/org/apache/nifi/documentation/html/ProcessorDocumentationWriterTest.java index faf66b58a9..5306ddf2ee 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/java/org/apache/nifi/documentation/html/ProcessorDocumentationWriterTest.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/java/org/apache/nifi/documentation/html/ProcessorDocumentationWriterTest.java @@ -25,6 +25,7 @@ import java.io.IOException; import org.apache.nifi.annotation.documentation.CapabilityDescription; import org.apache.nifi.documentation.DocumentationWriter; import org.apache.nifi.documentation.example.FullyDocumentedProcessor; +import org.apache.nifi.documentation.example.NakedProcessor; import org.apache.nifi.documentation.mock.MockProcessorInitializationContext; import org.junit.Test; @@ -40,8 +41,9 @@ public class ProcessorDocumentationWriterTest { ByteArrayOutputStream baos = new ByteArrayOutputStream(); writer.write(processor, baos, false); - + String results = new String(baos.toByteArray()); + XmlValidator.assertXmlValid(results); assertContains(results, FullyDocumentedProcessor.DIRECTORY.getDisplayName()); assertContains(results, FullyDocumentedProcessor.DIRECTORY.getDescription()); @@ -57,13 +59,45 @@ public class ProcessorDocumentationWriterTest { assertContains(results, FullyDocumentedProcessor.REL_SUCCESS.getDescription()); assertContains(results, FullyDocumentedProcessor.REL_FAILURE.getName()); assertContains(results, FullyDocumentedProcessor.REL_FAILURE.getDescription()); + assertContains(results, "Controller Service: "); + assertContains(results, "SampleService"); assertNotContains(results, "iconSecure.png"); - assertContains(results, FullyDocumentedProcessor.class.getAnnotation(CapabilityDescription.class).value()); + assertContains(results, FullyDocumentedProcessor.class.getAnnotation(CapabilityDescription.class) + .value()); assertNotContains(results, "This component has no required or optional properties."); assertNotContains(results, "No description provided."); assertNotContains(results, "No Tags provided."); assertNotContains(results, "Additional Details..."); } + @Test + public void testNakedProcessor() throws IOException { + NakedProcessor processor = new NakedProcessor(); + processor.initialize(new MockProcessorInitializationContext()); + + DocumentationWriter writer = new HtmlProcessorDocumentationWriter(); + + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + + writer.write(processor, baos, false); + + String results = new String(baos.toByteArray()); + XmlValidator.assertXmlValid(results); + + // no description + assertContains(results, "No description provided."); + + // no tags + assertContains(results, "None."); + + // properties + assertContains(results, "This component has no required or optional properties."); + + // relationships + assertContains(results, "This processor has no relationships."); + + + } + } From 22c6735126ed42dce33386e3ae31e0715d742c89 Mon Sep 17 00:00:00 2001 From: joewitt Date: Mon, 9 Mar 2015 21:10:11 -0400 Subject: [PATCH 14/26] NIFI-402-RC1 prepare release nifi-0.0.2-incubating-RC1 --- nifi/nifi-api/pom.xml | 2 +- nifi/nifi-assembly/pom.xml | 2 +- nifi/nifi-bootstrap/pom.xml | 2 +- .../nifi-data-provenance-utils/pom.xml | 2 +- .../nifi-expression-language/pom.xml | 2 +- .../nifi-flowfile-packager/pom.xml | 2 +- nifi/nifi-commons/nifi-logging-utils/pom.xml | 2 +- .../nifi-processor-utilities/pom.xml | 2 +- nifi/nifi-commons/nifi-properties/pom.xml | 2 +- nifi/nifi-commons/nifi-security-utils/pom.xml | 2 +- .../nifi-site-to-site-client/pom.xml | 8 +-- nifi/nifi-commons/nifi-socket-utils/pom.xml | 2 +- nifi/nifi-commons/nifi-utils/pom.xml | 4 +- nifi/nifi-commons/nifi-web-utils/pom.xml | 2 +- .../nifi-commons/nifi-write-ahead-log/pom.xml | 2 +- nifi/nifi-commons/pom.xml | 2 +- nifi/nifi-docs/pom.xml | 2 +- .../nifi-external/nifi-spark-receiver/pom.xml | 72 +++++++++---------- nifi/nifi-external/pom.xml | 2 +- .../nifi-processor-bundle-archetype/pom.xml | 6 +- nifi/nifi-maven-archetypes/pom.xml | 5 +- nifi/nifi-mock/pom.xml | 2 +- .../nifi-framework-nar/pom.xml | 2 +- .../nifi-administration/pom.xml | 2 +- .../nifi-framework/nifi-client-dto/pom.xml | 2 +- .../pom.xml | 2 +- .../nifi-cluster-protocol/pom.xml | 2 +- .../nifi-framework/nifi-cluster-web/pom.xml | 2 +- .../nifi-framework/nifi-cluster/pom.xml | 2 +- .../nifi-file-authorization-provider/pom.xml | 2 +- .../nifi-framework-core-api/pom.xml | 2 +- .../nifi-framework-core/pom.xml | 2 +- .../nifi-framework/nifi-nar-utils/pom.xml | 2 +- .../nifi-framework/nifi-resources/pom.xml | 2 +- .../nifi-framework/nifi-runtime/pom.xml | 2 +- .../nifi-framework/nifi-security/pom.xml | 2 +- .../nifi-framework/nifi-site-to-site/pom.xml | 2 +- .../nifi-framework/nifi-user-actions/pom.xml | 2 +- .../nifi-web/nifi-custom-ui-utilities/pom.xml | 2 +- .../nifi-web/nifi-jetty/pom.xml | 2 +- .../nifi-web/nifi-web-api/pom.xml | 4 +- .../nifi-web/nifi-web-docs/pom.xml | 2 +- .../nifi-web/nifi-web-error/pom.xml | 2 +- .../nifi-web-optimistic-locking/pom.xml | 2 +- .../nifi-web/nifi-web-security/pom.xml | 2 +- .../nifi-web/nifi-web-ui/pom.xml | 2 +- .../nifi-framework/nifi-web/pom.xml | 10 +-- .../nifi-framework/pom.xml | 2 +- .../nifi-framework-bundle/pom.xml | 34 ++++----- .../nifi-hadoop-nar/pom.xml | 2 +- .../nifi-hdfs-processors/pom.xml | 2 +- .../nifi-hadoop-bundle/pom.xml | 4 +- .../nifi-hadoop-libraries-nar/pom.xml | 5 +- .../nifi-hadoop-libraries-bundle/pom.xml | 2 +- .../nifi-jetty-bundle/pom.xml | 2 +- .../nifi-kafka-bundle/nifi-kafka-nar/pom.xml | 2 +- .../nifi-kafka-processors/pom.xml | 2 +- .../nifi-kafka-bundle/pom.xml | 4 +- .../nifi-kite-bundle/nifi-kite-nar/pom.xml | 2 +- .../nifi-kite-processors/pom.xml | 2 +- .../nifi-nar-bundles/nifi-kite-bundle/pom.xml | 4 +- .../pom.xml | 2 +- .../nifi-provenance-repository-nar/pom.xml | 2 +- .../pom.xml | 2 +- .../nifi-provenance-repository-bundle/pom.xml | 6 +- .../nifi-standard-nar/pom.xml | 2 +- .../nifi-standard-prioritizers/pom.xml | 2 +- .../nifi-standard-processors/pom.xml | 2 +- .../nifi-standard-reporting-tasks/pom.xml | 2 +- .../nifi-standard-bundle/pom.xml | 8 +-- .../pom.xml | 2 +- .../pom.xml | 2 +- .../nifi-distributed-cache-protocol/pom.xml | 2 +- .../nifi-distributed-cache-server/pom.xml | 2 +- .../pom.xml | 2 +- .../pom.xml | 2 +- .../nifi-http-context-map-api/pom.xml | 4 +- .../nifi-http-context-map-nar/pom.xml | 2 +- .../nifi-http-context-map/pom.xml | 2 +- .../nifi-http-context-map-bundle/pom.xml | 2 +- .../pom.xml | 2 +- .../nifi-ssl-context-nar/pom.xml | 2 +- .../nifi-ssl-context-service/pom.xml | 2 +- .../nifi-ssl-context-bundle/pom.xml | 2 +- .../nifi-ssl-context-service-api/pom.xml | 2 +- .../nifi-standard-services-api-nar/pom.xml | 2 +- .../nifi-standard-services/pom.xml | 2 +- .../nifi-update-attribute-model/pom.xml | 2 +- .../nifi-update-attribute-nar/pom.xml | 2 +- .../nifi-update-attribute-processor/pom.xml | 2 +- .../nifi-update-attribute-ui/pom.xml | 2 +- .../nifi-update-attribute-bundle/pom.xml | 8 +-- nifi/nifi-nar-bundles/pom.xml | 30 ++++---- nifi/pom.xml | 70 +++++++++--------- 94 files changed, 216 insertions(+), 222 deletions(-) diff --git a/nifi/nifi-api/pom.xml b/nifi/nifi-api/pom.xml index ec746a57ee..c8617bc039 100644 --- a/nifi/nifi-api/pom.xml +++ b/nifi/nifi-api/pom.xml @@ -18,7 +18,7 @@ org.apache.nifi nifi - 0.0.2-incubating-SNAPSHOT + 0.0.2-incubating nifi-api jar diff --git a/nifi/nifi-assembly/pom.xml b/nifi/nifi-assembly/pom.xml index 525eb4dab2..d73b3cfa5c 100644 --- a/nifi/nifi-assembly/pom.xml +++ b/nifi/nifi-assembly/pom.xml @@ -18,7 +18,7 @@ org.apache.nifi nifi - 0.0.2-incubating-SNAPSHOT + 0.0.2-incubating nifi-assembly pom diff --git a/nifi/nifi-bootstrap/pom.xml b/nifi/nifi-bootstrap/pom.xml index 12fa47333e..355e71f321 100644 --- a/nifi/nifi-bootstrap/pom.xml +++ b/nifi/nifi-bootstrap/pom.xml @@ -17,7 +17,7 @@ org.apache.nifi nifi - 0.0.2-incubating-SNAPSHOT + 0.0.2-incubating nifi-bootstrap jar diff --git a/nifi/nifi-commons/nifi-data-provenance-utils/pom.xml b/nifi/nifi-commons/nifi-data-provenance-utils/pom.xml index 580a8e7f1f..c04f251ff1 100644 --- a/nifi/nifi-commons/nifi-data-provenance-utils/pom.xml +++ b/nifi/nifi-commons/nifi-data-provenance-utils/pom.xml @@ -17,7 +17,7 @@ org.apache.nifi nifi-commons - 0.0.2-incubating-SNAPSHOT + 0.0.2-incubating nifi-data-provenance-utils jar diff --git a/nifi/nifi-commons/nifi-expression-language/pom.xml b/nifi/nifi-commons/nifi-expression-language/pom.xml index d41c788419..c1127c011b 100644 --- a/nifi/nifi-commons/nifi-expression-language/pom.xml +++ b/nifi/nifi-commons/nifi-expression-language/pom.xml @@ -17,7 +17,7 @@ org.apache.nifi nifi-commons - 0.0.2-incubating-SNAPSHOT + 0.0.2-incubating nifi-expression-language jar diff --git a/nifi/nifi-commons/nifi-flowfile-packager/pom.xml b/nifi/nifi-commons/nifi-flowfile-packager/pom.xml index 576780adc6..5164ea3d34 100644 --- a/nifi/nifi-commons/nifi-flowfile-packager/pom.xml +++ b/nifi/nifi-commons/nifi-flowfile-packager/pom.xml @@ -17,7 +17,7 @@ org.apache.nifi nifi-commons - 0.0.2-incubating-SNAPSHOT + 0.0.2-incubating nifi-flowfile-packager jar diff --git a/nifi/nifi-commons/nifi-logging-utils/pom.xml b/nifi/nifi-commons/nifi-logging-utils/pom.xml index b431254426..d26caba39d 100644 --- a/nifi/nifi-commons/nifi-logging-utils/pom.xml +++ b/nifi/nifi-commons/nifi-logging-utils/pom.xml @@ -18,7 +18,7 @@ org.apache.nifi nifi-commons - 0.0.2-incubating-SNAPSHOT + 0.0.2-incubating nifi-logging-utils Utilities for logging diff --git a/nifi/nifi-commons/nifi-processor-utilities/pom.xml b/nifi/nifi-commons/nifi-processor-utilities/pom.xml index 2e7318d887..fd9d43ec33 100644 --- a/nifi/nifi-commons/nifi-processor-utilities/pom.xml +++ b/nifi/nifi-commons/nifi-processor-utilities/pom.xml @@ -18,7 +18,7 @@ org.apache.nifi nifi-commons - 0.0.2-incubating-SNAPSHOT + 0.0.2-incubating nifi-processor-utils jar diff --git a/nifi/nifi-commons/nifi-properties/pom.xml b/nifi/nifi-commons/nifi-properties/pom.xml index 48c13213b6..4d06c7885a 100644 --- a/nifi/nifi-commons/nifi-properties/pom.xml +++ b/nifi/nifi-commons/nifi-properties/pom.xml @@ -18,7 +18,7 @@ org.apache.nifi nifi-commons - 0.0.2-incubating-SNAPSHOT + 0.0.2-incubating nifi-properties diff --git a/nifi/nifi-commons/nifi-security-utils/pom.xml b/nifi/nifi-commons/nifi-security-utils/pom.xml index b2b34b1ecd..24b7661d22 100644 --- a/nifi/nifi-commons/nifi-security-utils/pom.xml +++ b/nifi/nifi-commons/nifi-security-utils/pom.xml @@ -17,7 +17,7 @@ org.apache.nifi nifi-commons - 0.0.2-incubating-SNAPSHOT + 0.0.2-incubating nifi-security-utils Contains security functionality. diff --git a/nifi/nifi-commons/nifi-site-to-site-client/pom.xml b/nifi/nifi-commons/nifi-site-to-site-client/pom.xml index c440de20c3..aba8c54e4c 100644 --- a/nifi/nifi-commons/nifi-site-to-site-client/pom.xml +++ b/nifi/nifi-commons/nifi-site-to-site-client/pom.xml @@ -13,15 +13,13 @@ See the License for the specific language governing permissions and limitations under the License. --> - + 4.0.0 org.apache.nifi nifi-commons - 0.0.2-incubating-SNAPSHOT + 0.0.2-incubating nifi-site-to-site-client @@ -44,7 +42,7 @@ org.apache.nifi nifi-client-dto - 0.0.2-incubating-SNAPSHOT + 0.0.2-incubating diff --git a/nifi/nifi-commons/nifi-socket-utils/pom.xml b/nifi/nifi-commons/nifi-socket-utils/pom.xml index 06f710979a..b5865e7a29 100644 --- a/nifi/nifi-commons/nifi-socket-utils/pom.xml +++ b/nifi/nifi-commons/nifi-socket-utils/pom.xml @@ -18,7 +18,7 @@ org.apache.nifi nifi-commons - 0.0.2-incubating-SNAPSHOT + 0.0.2-incubating nifi-socket-utils Utilities for socket communication diff --git a/nifi/nifi-commons/nifi-utils/pom.xml b/nifi/nifi-commons/nifi-utils/pom.xml index 3613426fcd..904b618b12 100644 --- a/nifi/nifi-commons/nifi-utils/pom.xml +++ b/nifi/nifi-commons/nifi-utils/pom.xml @@ -18,10 +18,10 @@ org.apache.nifi nifi-commons - 0.0.2-incubating-SNAPSHOT + 0.0.2-incubating nifi-utils - 0.0.2-incubating-SNAPSHOT + 0.0.2-incubating jar - - 4.0.0 - - org.apache.nifi - nifi-external - 0.0.2-incubating-SNAPSHOT - - org.apache.nifi - nifi-spark-receiver - - - - org.apache.spark - spark-streaming_2.10 - 1.2.0 - - - org.apache.nifi - nifi-site-to-site-client - - + + + + 4.0.0 + + org.apache.nifi + nifi-external + 0.0.2-incubating + + org.apache.nifi + nifi-spark-receiver + + + + org.apache.spark + spark-streaming_2.10 + 1.2.0 + + + org.apache.nifi + nifi-site-to-site-client + + \ No newline at end of file diff --git a/nifi/nifi-external/pom.xml b/nifi/nifi-external/pom.xml index 878098f4a6..b7c6da5729 100644 --- a/nifi/nifi-external/pom.xml +++ b/nifi/nifi-external/pom.xml @@ -18,7 +18,7 @@ org.apache.nifi nifi - 0.0.2-incubating-SNAPSHOT + 0.0.2-incubating org.apache.nifi nifi-external diff --git a/nifi/nifi-maven-archetypes/nifi-processor-bundle-archetype/pom.xml b/nifi/nifi-maven-archetypes/nifi-processor-bundle-archetype/pom.xml index 540764caa0..28c3cb0c6b 100644 --- a/nifi/nifi-maven-archetypes/nifi-processor-bundle-archetype/pom.xml +++ b/nifi/nifi-maven-archetypes/nifi-processor-bundle-archetype/pom.xml @@ -13,14 +13,12 @@ See the License for the specific language governing permissions and limitations under the License. --> - + 4.0.0 org.apache.nifi nifi-maven-archetypes - 0.0.2-incubating-SNAPSHOT + 0.0.2-incubating nifi-processor-bundle-archetype diff --git a/nifi/nifi-maven-archetypes/pom.xml b/nifi/nifi-maven-archetypes/pom.xml index cb6fd00315..9ffea2589b 100644 --- a/nifi/nifi-maven-archetypes/pom.xml +++ b/nifi/nifi-maven-archetypes/pom.xml @@ -13,13 +13,12 @@ See the License for the specific language governing permissions and limitations under the License. --> - + 4.0.0 org.apache.nifi nifi - 0.0.2-incubating-SNAPSHOT + 0.0.2-incubating org.apache.nifi nifi-maven-archetypes diff --git a/nifi/nifi-mock/pom.xml b/nifi/nifi-mock/pom.xml index 2b1e534d11..0f6b484cbc 100644 --- a/nifi/nifi-mock/pom.xml +++ b/nifi/nifi-mock/pom.xml @@ -18,7 +18,7 @@ org.apache.nifi nifi - 0.0.2-incubating-SNAPSHOT + 0.0.2-incubating nifi-mock diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework-nar/pom.xml b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework-nar/pom.xml index 1b922706fc..8b4357bc34 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework-nar/pom.xml +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework-nar/pom.xml @@ -17,7 +17,7 @@ org.apache.nifi nifi-framework-bundle - 0.0.2-incubating-SNAPSHOT + 0.0.2-incubating nifi-framework-nar nar diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/pom.xml b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/pom.xml index 1f26280c81..5047a0c251 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/pom.xml +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/pom.xml @@ -18,7 +18,7 @@ org.apache.nifi nifi-framework - 0.0.2-incubating-SNAPSHOT + 0.0.2-incubating nifi-administration diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/pom.xml b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/pom.xml index 0cc3ddf965..ae8436a2f6 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/pom.xml +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/pom.xml @@ -18,7 +18,7 @@ org.apache.nifi nifi-framework - 0.0.2-incubating-SNAPSHOT + 0.0.2-incubating nifi-client-dto diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-authorization-provider/pom.xml b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-authorization-provider/pom.xml index e0a5d33c69..58198b201d 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-authorization-provider/pom.xml +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-authorization-provider/pom.xml @@ -18,7 +18,7 @@ org.apache.nifi nifi-framework - 0.0.2-incubating-SNAPSHOT + 0.0.2-incubating nifi-cluster-authorization-provider diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-protocol/pom.xml b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-protocol/pom.xml index afca65abf0..688b695b39 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-protocol/pom.xml +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-protocol/pom.xml @@ -18,7 +18,7 @@ org.apache.nifi nifi-framework - 0.0.2-incubating-SNAPSHOT + 0.0.2-incubating nifi-framework-cluster-protocol jar diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-web/pom.xml b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-web/pom.xml index a9e0c12e8b..3193bc25e1 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-web/pom.xml +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-web/pom.xml @@ -18,7 +18,7 @@ org.apache.nifi nifi-framework - 0.0.2-incubating-SNAPSHOT + 0.0.2-incubating nifi-framework-cluster-web jar diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/pom.xml b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/pom.xml index 3d9f3ace98..1e88c239c2 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/pom.xml +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/pom.xml @@ -18,7 +18,7 @@ org.apache.nifi nifi-framework - 0.0.2-incubating-SNAPSHOT + 0.0.2-incubating nifi-framework-cluster jar diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-file-authorization-provider/pom.xml b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-file-authorization-provider/pom.xml index 9cc4339675..ed7f2a4be2 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-file-authorization-provider/pom.xml +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-file-authorization-provider/pom.xml @@ -18,7 +18,7 @@ org.apache.nifi nifi-framework - 0.0.2-incubating-SNAPSHOT + 0.0.2-incubating nifi-file-authorization-provider diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/pom.xml b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/pom.xml index b3bb4469ec..4918c7cded 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/pom.xml +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/pom.xml @@ -18,7 +18,7 @@ org.apache.nifi nifi-framework - 0.0.2-incubating-SNAPSHOT + 0.0.2-incubating nifi-framework-core-api diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/pom.xml b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/pom.xml index 1ff5fbe90b..fc350392c0 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/pom.xml +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/pom.xml @@ -18,7 +18,7 @@ org.apache.nifi nifi-framework - 0.0.2-incubating-SNAPSHOT + 0.0.2-incubating nifi-framework-core jar diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/pom.xml b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/pom.xml index 272825c4da..f9ea0a5f50 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/pom.xml +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/pom.xml @@ -17,7 +17,7 @@ org.apache.nifi nifi-framework - 0.0.2-incubating-SNAPSHOT + 0.0.2-incubating nifi-nar-utils jar diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/pom.xml b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/pom.xml index fbe2067a69..2f1c1be944 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/pom.xml +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/pom.xml @@ -18,7 +18,7 @@ org.apache.nifi nifi-framework - 0.0.2-incubating-SNAPSHOT + 0.0.2-incubating nifi-resources pom diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-runtime/pom.xml b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-runtime/pom.xml index b75c35e6c0..73eb4770a0 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-runtime/pom.xml +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-runtime/pom.xml @@ -17,7 +17,7 @@ org.apache.nifi nifi-framework - 0.0.2-incubating-SNAPSHOT + 0.0.2-incubating nifi-runtime jar diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-security/pom.xml b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-security/pom.xml index bc01d8bc5e..1aabb4adc5 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-security/pom.xml +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-security/pom.xml @@ -17,7 +17,7 @@ org.apache.nifi nifi-framework - 0.0.2-incubating-SNAPSHOT + 0.0.2-incubating nifi-security Contains security functionality common to NiFi. diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/pom.xml b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/pom.xml index 2eba919ff0..c26c13daf9 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/pom.xml +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/pom.xml @@ -18,7 +18,7 @@ org.apache.nifi nifi-framework - 0.0.2-incubating-SNAPSHOT + 0.0.2-incubating nifi-site-to-site diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-user-actions/pom.xml b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-user-actions/pom.xml index 0bbd4ce9ef..97834ffb01 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-user-actions/pom.xml +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-user-actions/pom.xml @@ -18,7 +18,7 @@ org.apache.nifi nifi-framework - 0.0.2-incubating-SNAPSHOT + 0.0.2-incubating nifi-user-actions diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-custom-ui-utilities/pom.xml b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-custom-ui-utilities/pom.xml index f69b7a631d..1e81163f77 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-custom-ui-utilities/pom.xml +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-custom-ui-utilities/pom.xml @@ -18,7 +18,7 @@ org.apache.nifi nifi-web - 0.0.2-incubating-SNAPSHOT + 0.0.2-incubating nifi-custom-ui-utilities diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-jetty/pom.xml b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-jetty/pom.xml index f538576c35..95ea18e3d7 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-jetty/pom.xml +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-jetty/pom.xml @@ -18,7 +18,7 @@ org.apache.nifi nifi-web - 0.0.2-incubating-SNAPSHOT + 0.0.2-incubating nifi-jetty jar diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/pom.xml b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/pom.xml index 64a61fc428..5b36653f73 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/pom.xml +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/pom.xml @@ -18,7 +18,7 @@ org.apache.nifi nifi-web - 0.0.2-incubating-SNAPSHOT + 0.0.2-incubating org.apache.nifi nifi-web-api @@ -118,7 +118,7 @@ sources provided true - 0.0.2-incubating-SNAPSHOT + 0.0.2-incubating org.apache.nifi diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-docs/pom.xml b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-docs/pom.xml index 252c75cafb..0355a7c498 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-docs/pom.xml +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-docs/pom.xml @@ -17,7 +17,7 @@ org.apache.nifi nifi-web - 0.0.2-incubating-SNAPSHOT + 0.0.2-incubating org.apache.nifi nifi-web-docs diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-error/pom.xml b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-error/pom.xml index 6a77bd28c3..4d96ccdabb 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-error/pom.xml +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-error/pom.xml @@ -17,7 +17,7 @@ org.apache.nifi nifi-web - 0.0.2-incubating-SNAPSHOT + 0.0.2-incubating org.apache.nifi nifi-web-error diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-optimistic-locking/pom.xml b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-optimistic-locking/pom.xml index 0857d6f8de..60faeb2d65 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-optimistic-locking/pom.xml +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-optimistic-locking/pom.xml @@ -18,7 +18,7 @@ org.apache.nifi nifi-web - 0.0.2-incubating-SNAPSHOT + 0.0.2-incubating org.apache.nifi nifi-web-optimistic-locking diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/pom.xml b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/pom.xml index 9e5bab8025..a1c2adc91e 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/pom.xml +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/pom.xml @@ -18,7 +18,7 @@ org.apache.nifi nifi-web - 0.0.2-incubating-SNAPSHOT + 0.0.2-incubating org.apache.nifi nifi-web-security diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/pom.xml b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/pom.xml index f01cbea3ab..03819fa45c 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/pom.xml +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/pom.xml @@ -18,7 +18,7 @@ org.apache.nifi nifi-web - 0.0.2-incubating-SNAPSHOT + 0.0.2-incubating nifi-web-ui war diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/pom.xml b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/pom.xml index b776086986..3e975096f6 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/pom.xml +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/pom.xml @@ -18,7 +18,7 @@ org.apache.nifi nifi-framework - 0.0.2-incubating-SNAPSHOT + 0.0.2-incubating nifi-web pom @@ -38,25 +38,25 @@ org.apache.nifi nifi-web-api war - 0.0.2-incubating-SNAPSHOT + 0.0.2-incubating org.apache.nifi nifi-web-error war - 0.0.2-incubating-SNAPSHOT + 0.0.2-incubating org.apache.nifi nifi-web-docs war - 0.0.2-incubating-SNAPSHOT + 0.0.2-incubating org.apache.nifi nifi-web-ui war - 0.0.2-incubating-SNAPSHOT + 0.0.2-incubating diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/pom.xml b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/pom.xml index 30d6b0380c..9627cf92c7 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/pom.xml +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/pom.xml @@ -18,7 +18,7 @@ org.apache.nifi nifi-framework-bundle - 0.0.2-incubating-SNAPSHOT + 0.0.2-incubating nifi-framework pom diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/pom.xml b/nifi/nifi-nar-bundles/nifi-framework-bundle/pom.xml index d65771f3a7..18f365b7a3 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/pom.xml +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/pom.xml @@ -17,7 +17,7 @@ org.apache.nifi nifi-nar-bundles - 0.0.2-incubating-SNAPSHOT + 0.0.2-incubating nifi-framework-bundle pom @@ -31,82 +31,82 @@ org.apache.nifi nifi-framework-cluster-protocol - 0.0.2-incubating-SNAPSHOT + 0.0.2-incubating org.apache.nifi nifi-framework-cluster-web - 0.0.2-incubating-SNAPSHOT + 0.0.2-incubating org.apache.nifi nifi-file-authorization-provider - 0.0.2-incubating-SNAPSHOT + 0.0.2-incubating org.apache.nifi nifi-cluster-authorization-provider - 0.0.2-incubating-SNAPSHOT + 0.0.2-incubating org.apache.nifi nifi-framework-cluster - 0.0.2-incubating-SNAPSHOT + 0.0.2-incubating org.apache.nifi nifi-runtime - 0.0.2-incubating-SNAPSHOT + 0.0.2-incubating org.apache.nifi nifi-client-dto - 0.0.2-incubating-SNAPSHOT + 0.0.2-incubating org.apache.nifi nifi-security - 0.0.2-incubating-SNAPSHOT + 0.0.2-incubating org.apache.nifi nifi-framework-core-api - 0.0.2-incubating-SNAPSHOT + 0.0.2-incubating org.apache.nifi nifi-site-to-site - 0.0.2-incubating-SNAPSHOT + 0.0.2-incubating org.apache.nifi nifi-framework-core - 0.0.2-incubating-SNAPSHOT + 0.0.2-incubating org.apache.nifi nifi-user-actions - 0.0.2-incubating-SNAPSHOT + 0.0.2-incubating org.apache.nifi nifi-administration - 0.0.2-incubating-SNAPSHOT + 0.0.2-incubating org.apache.nifi nifi-jetty - 0.0.2-incubating-SNAPSHOT + 0.0.2-incubating org.apache.nifi nifi-web-optimistic-locking - 0.0.2-incubating-SNAPSHOT + 0.0.2-incubating org.apache.nifi nifi-web-security - 0.0.2-incubating-SNAPSHOT + 0.0.2-incubating diff --git a/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hadoop-nar/pom.xml b/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hadoop-nar/pom.xml index 5baf87ca04..4949e652b8 100644 --- a/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hadoop-nar/pom.xml +++ b/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hadoop-nar/pom.xml @@ -17,7 +17,7 @@ org.apache.nifi nifi-hadoop-bundle - 0.0.2-incubating-SNAPSHOT + 0.0.2-incubating nifi-hadoop-nar nar diff --git a/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/pom.xml b/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/pom.xml index d8f5006ccc..40ab0b508d 100644 --- a/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/pom.xml +++ b/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/pom.xml @@ -17,7 +17,7 @@ org.apache.nifi nifi-hadoop-bundle - 0.0.2-incubating-SNAPSHOT + 0.0.2-incubating nifi-hdfs-processors jar diff --git a/nifi/nifi-nar-bundles/nifi-hadoop-bundle/pom.xml b/nifi/nifi-nar-bundles/nifi-hadoop-bundle/pom.xml index 3c7680a3ea..8d06cba5e4 100644 --- a/nifi/nifi-nar-bundles/nifi-hadoop-bundle/pom.xml +++ b/nifi/nifi-nar-bundles/nifi-hadoop-bundle/pom.xml @@ -17,7 +17,7 @@ org.apache.nifi nifi-nar-bundles - 0.0.2-incubating-SNAPSHOT + 0.0.2-incubating nifi-hadoop-bundle pom @@ -31,7 +31,7 @@ org.apache.nifi nifi-hdfs-processors - 0.0.2-incubating-SNAPSHOT + 0.0.2-incubating diff --git a/nifi/nifi-nar-bundles/nifi-hadoop-libraries-bundle/nifi-hadoop-libraries-nar/pom.xml b/nifi/nifi-nar-bundles/nifi-hadoop-libraries-bundle/nifi-hadoop-libraries-nar/pom.xml index 174b97d431..49471c77d6 100644 --- a/nifi/nifi-nar-bundles/nifi-hadoop-libraries-bundle/nifi-hadoop-libraries-nar/pom.xml +++ b/nifi/nifi-nar-bundles/nifi-hadoop-libraries-bundle/nifi-hadoop-libraries-nar/pom.xml @@ -1,5 +1,4 @@ - + - + 4.0.0 org.apache.nifi nifi-standard-services - 0.0.2-incubating-SNAPSHOT + 0.0.2-incubating nifi-http-context-map-api diff --git a/nifi/nifi-nar-bundles/nifi-standard-services/nifi-http-context-map-bundle/nifi-http-context-map-nar/pom.xml b/nifi/nifi-nar-bundles/nifi-standard-services/nifi-http-context-map-bundle/nifi-http-context-map-nar/pom.xml index 653c218099..672c846cc4 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-services/nifi-http-context-map-bundle/nifi-http-context-map-nar/pom.xml +++ b/nifi/nifi-nar-bundles/nifi-standard-services/nifi-http-context-map-bundle/nifi-http-context-map-nar/pom.xml @@ -17,7 +17,7 @@ org.apache.nifi nifi-http-context-map-bundle - 0.0.2-incubating-SNAPSHOT + 0.0.2-incubating nifi-http-context-map-nar diff --git a/nifi/nifi-nar-bundles/nifi-standard-services/nifi-http-context-map-bundle/nifi-http-context-map/pom.xml b/nifi/nifi-nar-bundles/nifi-standard-services/nifi-http-context-map-bundle/nifi-http-context-map/pom.xml index abf78db8fa..f28c1e953e 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-services/nifi-http-context-map-bundle/nifi-http-context-map/pom.xml +++ b/nifi/nifi-nar-bundles/nifi-standard-services/nifi-http-context-map-bundle/nifi-http-context-map/pom.xml @@ -17,7 +17,7 @@ org.apache.nifi nifi-http-context-map-bundle - 0.0.2-incubating-SNAPSHOT + 0.0.2-incubating nifi-http-context-map diff --git a/nifi/nifi-nar-bundles/nifi-standard-services/nifi-http-context-map-bundle/pom.xml b/nifi/nifi-nar-bundles/nifi-standard-services/nifi-http-context-map-bundle/pom.xml index 5539bd8ed8..9e627e7e0e 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-services/nifi-http-context-map-bundle/pom.xml +++ b/nifi/nifi-nar-bundles/nifi-standard-services/nifi-http-context-map-bundle/pom.xml @@ -17,7 +17,7 @@ org.apache.nifi nifi-standard-services - 0.0.2-incubating-SNAPSHOT + 0.0.2-incubating nifi-http-context-map-bundle diff --git a/nifi/nifi-nar-bundles/nifi-standard-services/nifi-load-distribution-service-api/pom.xml b/nifi/nifi-nar-bundles/nifi-standard-services/nifi-load-distribution-service-api/pom.xml index 079ac60a58..76a03709e3 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-services/nifi-load-distribution-service-api/pom.xml +++ b/nifi/nifi-nar-bundles/nifi-standard-services/nifi-load-distribution-service-api/pom.xml @@ -17,7 +17,7 @@ org.apache.nifi nifi-standard-services - 0.0.2-incubating-SNAPSHOT + 0.0.2-incubating nifi-load-distribution-service-api jar diff --git a/nifi/nifi-nar-bundles/nifi-standard-services/nifi-ssl-context-bundle/nifi-ssl-context-nar/pom.xml b/nifi/nifi-nar-bundles/nifi-standard-services/nifi-ssl-context-bundle/nifi-ssl-context-nar/pom.xml index a9d62d68fc..a0cf7e55b8 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-services/nifi-ssl-context-bundle/nifi-ssl-context-nar/pom.xml +++ b/nifi/nifi-nar-bundles/nifi-standard-services/nifi-ssl-context-bundle/nifi-ssl-context-nar/pom.xml @@ -17,7 +17,7 @@ org.apache.nifi nifi-ssl-context-bundle - 0.0.2-incubating-SNAPSHOT + 0.0.2-incubating nifi-ssl-context-service-nar nar diff --git a/nifi/nifi-nar-bundles/nifi-standard-services/nifi-ssl-context-bundle/nifi-ssl-context-service/pom.xml b/nifi/nifi-nar-bundles/nifi-standard-services/nifi-ssl-context-bundle/nifi-ssl-context-service/pom.xml index 498c095d6b..44f26eecfa 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-services/nifi-ssl-context-bundle/nifi-ssl-context-service/pom.xml +++ b/nifi/nifi-nar-bundles/nifi-standard-services/nifi-ssl-context-bundle/nifi-ssl-context-service/pom.xml @@ -17,7 +17,7 @@ org.apache.nifi nifi-ssl-context-bundle - 0.0.2-incubating-SNAPSHOT + 0.0.2-incubating nifi-ssl-context-service jar diff --git a/nifi/nifi-nar-bundles/nifi-standard-services/nifi-ssl-context-bundle/pom.xml b/nifi/nifi-nar-bundles/nifi-standard-services/nifi-ssl-context-bundle/pom.xml index 67d84a4337..ffd7f2a209 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-services/nifi-ssl-context-bundle/pom.xml +++ b/nifi/nifi-nar-bundles/nifi-standard-services/nifi-ssl-context-bundle/pom.xml @@ -17,7 +17,7 @@ org.apache.nifi nifi-standard-services - 0.0.2-incubating-SNAPSHOT + 0.0.2-incubating nifi-ssl-context-bundle pom diff --git a/nifi/nifi-nar-bundles/nifi-standard-services/nifi-ssl-context-service-api/pom.xml b/nifi/nifi-nar-bundles/nifi-standard-services/nifi-ssl-context-service-api/pom.xml index 9ed6d0ee51..de343e5e63 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-services/nifi-ssl-context-service-api/pom.xml +++ b/nifi/nifi-nar-bundles/nifi-standard-services/nifi-ssl-context-service-api/pom.xml @@ -17,7 +17,7 @@ org.apache.nifi nifi-standard-services - 0.0.2-incubating-SNAPSHOT + 0.0.2-incubating nifi-ssl-context-service-api jar diff --git a/nifi/nifi-nar-bundles/nifi-standard-services/nifi-standard-services-api-nar/pom.xml b/nifi/nifi-nar-bundles/nifi-standard-services/nifi-standard-services-api-nar/pom.xml index 7668cda31f..60a6a8c4a0 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-services/nifi-standard-services-api-nar/pom.xml +++ b/nifi/nifi-nar-bundles/nifi-standard-services/nifi-standard-services-api-nar/pom.xml @@ -17,7 +17,7 @@ org.apache.nifi nifi-standard-services - 0.0.2-incubating-SNAPSHOT + 0.0.2-incubating nifi-standard-services-api-nar nar diff --git a/nifi/nifi-nar-bundles/nifi-standard-services/pom.xml b/nifi/nifi-nar-bundles/nifi-standard-services/pom.xml index 47762b3673..9d3264767d 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-services/pom.xml +++ b/nifi/nifi-nar-bundles/nifi-standard-services/pom.xml @@ -18,7 +18,7 @@ org.apache.nifi nifi-nar-bundles - 0.0.2-incubating-SNAPSHOT + 0.0.2-incubating nifi-standard-services pom diff --git a/nifi/nifi-nar-bundles/nifi-update-attribute-bundle/nifi-update-attribute-model/pom.xml b/nifi/nifi-nar-bundles/nifi-update-attribute-bundle/nifi-update-attribute-model/pom.xml index 9d703432ba..493bac2405 100644 --- a/nifi/nifi-nar-bundles/nifi-update-attribute-bundle/nifi-update-attribute-model/pom.xml +++ b/nifi/nifi-nar-bundles/nifi-update-attribute-bundle/nifi-update-attribute-model/pom.xml @@ -18,7 +18,7 @@ org.apache.nifi nifi-update-attribute-bundle - 0.0.2-incubating-SNAPSHOT + 0.0.2-incubating org.apache.nifi nifi-update-attribute-model diff --git a/nifi/nifi-nar-bundles/nifi-update-attribute-bundle/nifi-update-attribute-nar/pom.xml b/nifi/nifi-nar-bundles/nifi-update-attribute-bundle/nifi-update-attribute-nar/pom.xml index 89762eaccf..530281f373 100644 --- a/nifi/nifi-nar-bundles/nifi-update-attribute-bundle/nifi-update-attribute-nar/pom.xml +++ b/nifi/nifi-nar-bundles/nifi-update-attribute-bundle/nifi-update-attribute-nar/pom.xml @@ -17,7 +17,7 @@ org.apache.nifi nifi-update-attribute-bundle - 0.0.2-incubating-SNAPSHOT + 0.0.2-incubating nifi-update-attribute-nar nar diff --git a/nifi/nifi-nar-bundles/nifi-update-attribute-bundle/nifi-update-attribute-processor/pom.xml b/nifi/nifi-nar-bundles/nifi-update-attribute-bundle/nifi-update-attribute-processor/pom.xml index deb787f554..ed348f8aee 100644 --- a/nifi/nifi-nar-bundles/nifi-update-attribute-bundle/nifi-update-attribute-processor/pom.xml +++ b/nifi/nifi-nar-bundles/nifi-update-attribute-bundle/nifi-update-attribute-processor/pom.xml @@ -18,7 +18,7 @@ org.apache.nifi nifi-update-attribute-bundle - 0.0.2-incubating-SNAPSHOT + 0.0.2-incubating org.apache.nifi nifi-update-attribute-processor diff --git a/nifi/nifi-nar-bundles/nifi-update-attribute-bundle/nifi-update-attribute-ui/pom.xml b/nifi/nifi-nar-bundles/nifi-update-attribute-bundle/nifi-update-attribute-ui/pom.xml index 04b9ad6047..59191c8aed 100644 --- a/nifi/nifi-nar-bundles/nifi-update-attribute-bundle/nifi-update-attribute-ui/pom.xml +++ b/nifi/nifi-nar-bundles/nifi-update-attribute-bundle/nifi-update-attribute-ui/pom.xml @@ -18,7 +18,7 @@ org.apache.nifi nifi-update-attribute-bundle - 0.0.2-incubating-SNAPSHOT + 0.0.2-incubating nifi-update-attribute-ui war diff --git a/nifi/nifi-nar-bundles/nifi-update-attribute-bundle/pom.xml b/nifi/nifi-nar-bundles/nifi-update-attribute-bundle/pom.xml index aac3685d75..1ed558a7ab 100644 --- a/nifi/nifi-nar-bundles/nifi-update-attribute-bundle/pom.xml +++ b/nifi/nifi-nar-bundles/nifi-update-attribute-bundle/pom.xml @@ -18,7 +18,7 @@ org.apache.nifi nifi-nar-bundles - 0.0.2-incubating-SNAPSHOT + 0.0.2-incubating nifi-update-attribute-bundle pom @@ -34,18 +34,18 @@ org.apache.nifi nifi-update-attribute-model - 0.0.2-incubating-SNAPSHOT + 0.0.2-incubating org.apache.nifi nifi-update-attribute-processor - 0.0.2-incubating-SNAPSHOT + 0.0.2-incubating org.apache.nifi nifi-update-attribute-ui war - 0.0.2-incubating-SNAPSHOT + 0.0.2-incubating diff --git a/nifi/nifi-nar-bundles/pom.xml b/nifi/nifi-nar-bundles/pom.xml index 289035149a..7b8c21afde 100644 --- a/nifi/nifi-nar-bundles/pom.xml +++ b/nifi/nifi-nar-bundles/pom.xml @@ -19,7 +19,7 @@ org.apache.nifi nifi - 0.0.2-incubating-SNAPSHOT + 0.0.2-incubating org.apache.nifi nifi-nar-bundles @@ -41,81 +41,81 @@ org.apache.nifi nifi-distributed-cache-client-service - 0.0.2-incubating-SNAPSHOT + 0.0.2-incubating org.apache.nifi nifi-distributed-cache-client-service-api - 0.0.2-incubating-SNAPSHOT + 0.0.2-incubating provided org.apache.nifi nifi-ssl-context-service-api - 0.0.2-incubating-SNAPSHOT + 0.0.2-incubating provided org.apache.nifi nifi-load-distribution-service-api - 0.0.2-incubating-SNAPSHOT + 0.0.2-incubating provided org.apache.nifi nifi-http-context-map-api - 0.0.2-incubating-SNAPSHOT + 0.0.2-incubating provided org.apache.nifi nifi-distributed-cache-protocol - 0.0.2-incubating-SNAPSHOT + 0.0.2-incubating org.apache.nifi nifi-distributed-cache-server - 0.0.2-incubating-SNAPSHOT + 0.0.2-incubating org.apache.nifi nifi-ssl-context-service - 0.0.2-incubating-SNAPSHOT + 0.0.2-incubating org.apache.nifi nifi-http-context-map - 0.0.2-incubating-SNAPSHOT + 0.0.2-incubating org.apache.nifi nifi-volatile-provenance-repository - 0.0.2-incubating-SNAPSHOT + 0.0.2-incubating test org.apache.nifi nifi-api - 0.0.2-incubating-SNAPSHOT + 0.0.2-incubating provided org.apache.nifi nifi-runtime - 0.0.2-incubating-SNAPSHOT + 0.0.2-incubating provided org.apache.nifi nifi-nar-utils - 0.0.2-incubating-SNAPSHOT + 0.0.2-incubating provided org.apache.nifi nifi-properties - 0.0.2-incubating-SNAPSHOT + 0.0.2-incubating provided diff --git a/nifi/pom.xml b/nifi/pom.xml index bd2aa4d0c4..ff6f6ec8de 100644 --- a/nifi/pom.xml +++ b/nifi/pom.xml @@ -23,7 +23,7 @@ org.apache.nifi nifi - 0.0.2-incubating-SNAPSHOT + 0.0.2-incubating pom Apache NiFi(incubating) is an easy to use, powerful, and reliable system to process and distribute data. http://nifi.incubator.apache.org @@ -71,7 +71,7 @@ scm:git:git://git.apache.org/incubator-nifi.git scm:git:https://git-wip-us.apache.org/repos/asf/incubator-nifi.git https://git-wip-us.apache.org/repos/asf?p=incubator-nifi.git - HEAD + nifi-0.0.2-incubating-RC1 JIRA @@ -647,62 +647,62 @@ org.apache.nifi nifi-api - 0.0.2-incubating-SNAPSHOT + 0.0.2-incubating org.apache.nifi nifi-utils - 0.0.2-incubating-SNAPSHOT + 0.0.2-incubating org.apache.nifi nifi-site-to-site-client - 0.0.2-incubating-SNAPSHOT + 0.0.2-incubating org.apache.nifi nifi-web-utils - 0.0.2-incubating-SNAPSHOT + 0.0.2-incubating org.apache.nifi nifi-expression-language - 0.0.2-incubating-SNAPSHOT + 0.0.2-incubating org.apache.nifi nifi-custom-ui-utilities - 0.0.2-incubating-SNAPSHOT + 0.0.2-incubating org.apache.nifi nifi-flowfile-packager - 0.0.2-incubating-SNAPSHOT + 0.0.2-incubating org.apache.nifi nifi-socket-utils - 0.0.2-incubating-SNAPSHOT + 0.0.2-incubating org.apache.nifi nifi-data-provenance-utils - 0.0.2-incubating-SNAPSHOT + 0.0.2-incubating org.apache.nifi nifi-runtime - 0.0.2-incubating-SNAPSHOT + 0.0.2-incubating org.apache.nifi nifi-bootstrap - 0.0.2-incubating-SNAPSHOT + 0.0.2-incubating org.apache.nifi nifi-resources - 0.0.2-incubating-SNAPSHOT + 0.0.2-incubating resources runtime zip @@ -710,7 +710,7 @@ org.apache.nifi nifi-docs - 0.0.2-incubating-SNAPSHOT + 0.0.2-incubating resources runtime zip @@ -718,116 +718,116 @@ org.apache.nifi nifi-framework-nar - 0.0.2-incubating-SNAPSHOT + 0.0.2-incubating nar org.apache.nifi nifi-provenance-repository-nar - 0.0.2-incubating-SNAPSHOT + 0.0.2-incubating nar org.apache.nifi nifi-standard-services-api-nar - 0.0.2-incubating-SNAPSHOT + 0.0.2-incubating nar org.apache.nifi nifi-ssl-context-service-nar - 0.0.2-incubating-SNAPSHOT + 0.0.2-incubating nar org.apache.nifi nifi-distributed-cache-services-nar - 0.0.2-incubating-SNAPSHOT + 0.0.2-incubating nar org.apache.nifi nifi-standard-nar - 0.0.2-incubating-SNAPSHOT + 0.0.2-incubating nar org.apache.nifi nifi-jetty-bundle - 0.0.2-incubating-SNAPSHOT + 0.0.2-incubating nar org.apache.nifi nifi-update-attribute-nar - 0.0.2-incubating-SNAPSHOT + 0.0.2-incubating nar org.apache.nifi nifi-hadoop-libraries-nar - 0.0.2-incubating-SNAPSHOT + 0.0.2-incubating nar org.apache.nifi nifi-hadoop-nar - 0.0.2-incubating-SNAPSHOT + 0.0.2-incubating nar org.apache.nifi nifi-kite-nar - 0.0.2-incubating-SNAPSHOT + 0.0.2-incubating nar org.apache.nifi nifi-kafka-nar - 0.0.2-incubating-SNAPSHOT + 0.0.2-incubating nar org.apache.nifi nifi-http-context-map-nar - 0.0.2-incubating-SNAPSHOT + 0.0.2-incubating nar org.apache.nifi nifi-properties - 0.0.2-incubating-SNAPSHOT + 0.0.2-incubating org.apache.nifi nifi-security-utils - 0.0.2-incubating-SNAPSHOT + 0.0.2-incubating org.apache.nifi nifi-logging-utils - 0.0.2-incubating-SNAPSHOT + 0.0.2-incubating org.apache.nifi nifi-nar-utils - 0.0.2-incubating-SNAPSHOT + 0.0.2-incubating org.apache.nifi nifi-processor-utils - 0.0.2-incubating-SNAPSHOT + 0.0.2-incubating org.apache.nifi nifi-mock - 0.0.2-incubating-SNAPSHOT + 0.0.2-incubating test org.apache.nifi nifi-write-ahead-log - 0.0.2-incubating-SNAPSHOT + 0.0.2-incubating com.jayway.jsonpath From 171dae3c8f0ecc77f934f0e5146e4782bc282fb5 Mon Sep 17 00:00:00 2001 From: joewitt Date: Mon, 9 Mar 2015 21:10:22 -0400 Subject: [PATCH 15/26] NIFI-402-RC1 prepare for next development iteration --- nifi/nifi-api/pom.xml | 2 +- nifi/nifi-assembly/pom.xml | 2 +- nifi/nifi-bootstrap/pom.xml | 2 +- .../nifi-data-provenance-utils/pom.xml | 2 +- .../nifi-expression-language/pom.xml | 2 +- .../nifi-flowfile-packager/pom.xml | 2 +- nifi/nifi-commons/nifi-logging-utils/pom.xml | 2 +- .../nifi-processor-utilities/pom.xml | 2 +- nifi/nifi-commons/nifi-properties/pom.xml | 2 +- nifi/nifi-commons/nifi-security-utils/pom.xml | 2 +- .../nifi-site-to-site-client/pom.xml | 4 +- nifi/nifi-commons/nifi-socket-utils/pom.xml | 2 +- nifi/nifi-commons/nifi-utils/pom.xml | 4 +- nifi/nifi-commons/nifi-web-utils/pom.xml | 2 +- .../nifi-commons/nifi-write-ahead-log/pom.xml | 2 +- nifi/nifi-commons/pom.xml | 2 +- nifi/nifi-docs/pom.xml | 2 +- .../nifi-external/nifi-spark-receiver/pom.xml | 2 +- nifi/nifi-external/pom.xml | 2 +- .../nifi-processor-bundle-archetype/pom.xml | 2 +- nifi/nifi-maven-archetypes/pom.xml | 2 +- nifi/nifi-mock/pom.xml | 2 +- .../nifi-framework-nar/pom.xml | 2 +- .../nifi-administration/pom.xml | 2 +- .../nifi-framework/nifi-client-dto/pom.xml | 2 +- .../pom.xml | 2 +- .../nifi-cluster-protocol/pom.xml | 2 +- .../nifi-framework/nifi-cluster-web/pom.xml | 2 +- .../nifi-framework/nifi-cluster/pom.xml | 2 +- .../nifi-file-authorization-provider/pom.xml | 2 +- .../nifi-framework-core-api/pom.xml | 2 +- .../nifi-framework-core/pom.xml | 2 +- .../nifi-framework/nifi-nar-utils/pom.xml | 2 +- .../nifi-framework/nifi-resources/pom.xml | 2 +- .../nifi-framework/nifi-runtime/pom.xml | 2 +- .../nifi-framework/nifi-security/pom.xml | 2 +- .../nifi-framework/nifi-site-to-site/pom.xml | 2 +- .../nifi-framework/nifi-user-actions/pom.xml | 2 +- .../nifi-web/nifi-custom-ui-utilities/pom.xml | 2 +- .../nifi-web/nifi-jetty/pom.xml | 2 +- .../nifi-web/nifi-web-api/pom.xml | 4 +- .../nifi-web/nifi-web-docs/pom.xml | 2 +- .../nifi-web/nifi-web-error/pom.xml | 2 +- .../nifi-web-optimistic-locking/pom.xml | 2 +- .../nifi-web/nifi-web-security/pom.xml | 2 +- .../nifi-web/nifi-web-ui/pom.xml | 2 +- .../nifi-framework/nifi-web/pom.xml | 10 +-- .../nifi-framework/pom.xml | 2 +- .../nifi-framework-bundle/pom.xml | 34 ++++----- .../nifi-hadoop-nar/pom.xml | 2 +- .../nifi-hdfs-processors/pom.xml | 2 +- .../nifi-hadoop-bundle/pom.xml | 4 +- .../nifi-hadoop-libraries-nar/pom.xml | 2 +- .../nifi-hadoop-libraries-bundle/pom.xml | 2 +- .../nifi-jetty-bundle/pom.xml | 2 +- .../nifi-kafka-bundle/nifi-kafka-nar/pom.xml | 2 +- .../nifi-kafka-processors/pom.xml | 2 +- .../nifi-kafka-bundle/pom.xml | 4 +- .../nifi-kite-bundle/nifi-kite-nar/pom.xml | 2 +- .../nifi-kite-processors/pom.xml | 2 +- .../nifi-nar-bundles/nifi-kite-bundle/pom.xml | 4 +- .../pom.xml | 2 +- .../nifi-provenance-repository-nar/pom.xml | 2 +- .../pom.xml | 2 +- .../nifi-provenance-repository-bundle/pom.xml | 6 +- .../nifi-standard-nar/pom.xml | 2 +- .../nifi-standard-prioritizers/pom.xml | 2 +- .../nifi-standard-processors/pom.xml | 2 +- .../nifi-standard-reporting-tasks/pom.xml | 2 +- .../nifi-standard-bundle/pom.xml | 8 +-- .../pom.xml | 2 +- .../pom.xml | 2 +- .../nifi-distributed-cache-protocol/pom.xml | 2 +- .../nifi-distributed-cache-server/pom.xml | 2 +- .../pom.xml | 2 +- .../pom.xml | 2 +- .../nifi-http-context-map-api/pom.xml | 2 +- .../nifi-http-context-map-nar/pom.xml | 2 +- .../nifi-http-context-map/pom.xml | 2 +- .../nifi-http-context-map-bundle/pom.xml | 2 +- .../pom.xml | 2 +- .../nifi-ssl-context-nar/pom.xml | 2 +- .../nifi-ssl-context-service/pom.xml | 2 +- .../nifi-ssl-context-bundle/pom.xml | 2 +- .../nifi-ssl-context-service-api/pom.xml | 2 +- .../nifi-standard-services-api-nar/pom.xml | 2 +- .../nifi-standard-services/pom.xml | 2 +- .../nifi-update-attribute-model/pom.xml | 2 +- .../nifi-update-attribute-nar/pom.xml | 2 +- .../nifi-update-attribute-processor/pom.xml | 2 +- .../nifi-update-attribute-ui/pom.xml | 2 +- .../nifi-update-attribute-bundle/pom.xml | 8 +-- nifi/nifi-nar-bundles/pom.xml | 30 ++++---- nifi/pom.xml | 70 +++++++++---------- 94 files changed, 176 insertions(+), 176 deletions(-) diff --git a/nifi/nifi-api/pom.xml b/nifi/nifi-api/pom.xml index c8617bc039..6105671e9b 100644 --- a/nifi/nifi-api/pom.xml +++ b/nifi/nifi-api/pom.xml @@ -18,7 +18,7 @@ org.apache.nifi nifi - 0.0.2-incubating + 0.0.3-incubating-SNAPSHOT nifi-api jar diff --git a/nifi/nifi-assembly/pom.xml b/nifi/nifi-assembly/pom.xml index d73b3cfa5c..200a9e0b8f 100644 --- a/nifi/nifi-assembly/pom.xml +++ b/nifi/nifi-assembly/pom.xml @@ -18,7 +18,7 @@ org.apache.nifi nifi - 0.0.2-incubating + 0.0.3-incubating-SNAPSHOT nifi-assembly pom diff --git a/nifi/nifi-bootstrap/pom.xml b/nifi/nifi-bootstrap/pom.xml index 355e71f321..b2188a8efa 100644 --- a/nifi/nifi-bootstrap/pom.xml +++ b/nifi/nifi-bootstrap/pom.xml @@ -17,7 +17,7 @@ org.apache.nifi nifi - 0.0.2-incubating + 0.0.3-incubating-SNAPSHOT nifi-bootstrap jar diff --git a/nifi/nifi-commons/nifi-data-provenance-utils/pom.xml b/nifi/nifi-commons/nifi-data-provenance-utils/pom.xml index c04f251ff1..a7cdf930d9 100644 --- a/nifi/nifi-commons/nifi-data-provenance-utils/pom.xml +++ b/nifi/nifi-commons/nifi-data-provenance-utils/pom.xml @@ -17,7 +17,7 @@ org.apache.nifi nifi-commons - 0.0.2-incubating + 0.0.3-incubating-SNAPSHOT nifi-data-provenance-utils jar diff --git a/nifi/nifi-commons/nifi-expression-language/pom.xml b/nifi/nifi-commons/nifi-expression-language/pom.xml index c1127c011b..e94f1f6eb6 100644 --- a/nifi/nifi-commons/nifi-expression-language/pom.xml +++ b/nifi/nifi-commons/nifi-expression-language/pom.xml @@ -17,7 +17,7 @@ org.apache.nifi nifi-commons - 0.0.2-incubating + 0.0.3-incubating-SNAPSHOT nifi-expression-language jar diff --git a/nifi/nifi-commons/nifi-flowfile-packager/pom.xml b/nifi/nifi-commons/nifi-flowfile-packager/pom.xml index 5164ea3d34..5543d1c3eb 100644 --- a/nifi/nifi-commons/nifi-flowfile-packager/pom.xml +++ b/nifi/nifi-commons/nifi-flowfile-packager/pom.xml @@ -17,7 +17,7 @@ org.apache.nifi nifi-commons - 0.0.2-incubating + 0.0.3-incubating-SNAPSHOT nifi-flowfile-packager jar diff --git a/nifi/nifi-commons/nifi-logging-utils/pom.xml b/nifi/nifi-commons/nifi-logging-utils/pom.xml index d26caba39d..99ab542b80 100644 --- a/nifi/nifi-commons/nifi-logging-utils/pom.xml +++ b/nifi/nifi-commons/nifi-logging-utils/pom.xml @@ -18,7 +18,7 @@ org.apache.nifi nifi-commons - 0.0.2-incubating + 0.0.3-incubating-SNAPSHOT nifi-logging-utils Utilities for logging diff --git a/nifi/nifi-commons/nifi-processor-utilities/pom.xml b/nifi/nifi-commons/nifi-processor-utilities/pom.xml index fd9d43ec33..e3b684fe05 100644 --- a/nifi/nifi-commons/nifi-processor-utilities/pom.xml +++ b/nifi/nifi-commons/nifi-processor-utilities/pom.xml @@ -18,7 +18,7 @@ org.apache.nifi nifi-commons - 0.0.2-incubating + 0.0.3-incubating-SNAPSHOT nifi-processor-utils jar diff --git a/nifi/nifi-commons/nifi-properties/pom.xml b/nifi/nifi-commons/nifi-properties/pom.xml index 4d06c7885a..6bfd82eef9 100644 --- a/nifi/nifi-commons/nifi-properties/pom.xml +++ b/nifi/nifi-commons/nifi-properties/pom.xml @@ -18,7 +18,7 @@ org.apache.nifi nifi-commons - 0.0.2-incubating + 0.0.3-incubating-SNAPSHOT nifi-properties diff --git a/nifi/nifi-commons/nifi-security-utils/pom.xml b/nifi/nifi-commons/nifi-security-utils/pom.xml index 24b7661d22..7bf1b4aecc 100644 --- a/nifi/nifi-commons/nifi-security-utils/pom.xml +++ b/nifi/nifi-commons/nifi-security-utils/pom.xml @@ -17,7 +17,7 @@ org.apache.nifi nifi-commons - 0.0.2-incubating + 0.0.3-incubating-SNAPSHOT nifi-security-utils Contains security functionality. diff --git a/nifi/nifi-commons/nifi-site-to-site-client/pom.xml b/nifi/nifi-commons/nifi-site-to-site-client/pom.xml index aba8c54e4c..76bfd4cb11 100644 --- a/nifi/nifi-commons/nifi-site-to-site-client/pom.xml +++ b/nifi/nifi-commons/nifi-site-to-site-client/pom.xml @@ -19,7 +19,7 @@ org.apache.nifi nifi-commons - 0.0.2-incubating + 0.0.3-incubating-SNAPSHOT nifi-site-to-site-client @@ -42,7 +42,7 @@ org.apache.nifi nifi-client-dto - 0.0.2-incubating + 0.0.3-incubating-SNAPSHOT diff --git a/nifi/nifi-commons/nifi-socket-utils/pom.xml b/nifi/nifi-commons/nifi-socket-utils/pom.xml index b5865e7a29..f2f3397c94 100644 --- a/nifi/nifi-commons/nifi-socket-utils/pom.xml +++ b/nifi/nifi-commons/nifi-socket-utils/pom.xml @@ -18,7 +18,7 @@ org.apache.nifi nifi-commons - 0.0.2-incubating + 0.0.3-incubating-SNAPSHOT nifi-socket-utils Utilities for socket communication diff --git a/nifi/nifi-commons/nifi-utils/pom.xml b/nifi/nifi-commons/nifi-utils/pom.xml index 904b618b12..86fd309bfd 100644 --- a/nifi/nifi-commons/nifi-utils/pom.xml +++ b/nifi/nifi-commons/nifi-utils/pom.xml @@ -18,10 +18,10 @@ org.apache.nifi nifi-commons - 0.0.2-incubating + 0.0.3-incubating-SNAPSHOT nifi-utils - 0.0.2-incubating + 0.0.3-incubating-SNAPSHOT jar org.apache.nifi nifi-api - 0.0.2-incubating + 0.0.3-incubating-SNAPSHOT provided org.apache.nifi nifi-runtime - 0.0.2-incubating + 0.0.3-incubating-SNAPSHOT provided org.apache.nifi nifi-nar-utils - 0.0.2-incubating + 0.0.3-incubating-SNAPSHOT provided org.apache.nifi nifi-properties - 0.0.2-incubating + 0.0.3-incubating-SNAPSHOT provided diff --git a/nifi/pom.xml b/nifi/pom.xml index ff6f6ec8de..09f08d1dcb 100644 --- a/nifi/pom.xml +++ b/nifi/pom.xml @@ -23,7 +23,7 @@ org.apache.nifi nifi - 0.0.2-incubating + 0.0.3-incubating-SNAPSHOT pom Apache NiFi(incubating) is an easy to use, powerful, and reliable system to process and distribute data. http://nifi.incubator.apache.org @@ -71,7 +71,7 @@ scm:git:git://git.apache.org/incubator-nifi.git scm:git:https://git-wip-us.apache.org/repos/asf/incubator-nifi.git https://git-wip-us.apache.org/repos/asf?p=incubator-nifi.git - nifi-0.0.2-incubating-RC1 + HEAD JIRA @@ -647,62 +647,62 @@ org.apache.nifi nifi-api - 0.0.2-incubating + 0.0.3-incubating-SNAPSHOT org.apache.nifi nifi-utils - 0.0.2-incubating + 0.0.3-incubating-SNAPSHOT org.apache.nifi nifi-site-to-site-client - 0.0.2-incubating + 0.0.3-incubating-SNAPSHOT org.apache.nifi nifi-web-utils - 0.0.2-incubating + 0.0.3-incubating-SNAPSHOT org.apache.nifi nifi-expression-language - 0.0.2-incubating + 0.0.3-incubating-SNAPSHOT org.apache.nifi nifi-custom-ui-utilities - 0.0.2-incubating + 0.0.3-incubating-SNAPSHOT org.apache.nifi nifi-flowfile-packager - 0.0.2-incubating + 0.0.3-incubating-SNAPSHOT org.apache.nifi nifi-socket-utils - 0.0.2-incubating + 0.0.3-incubating-SNAPSHOT org.apache.nifi nifi-data-provenance-utils - 0.0.2-incubating + 0.0.3-incubating-SNAPSHOT org.apache.nifi nifi-runtime - 0.0.2-incubating + 0.0.3-incubating-SNAPSHOT org.apache.nifi nifi-bootstrap - 0.0.2-incubating + 0.0.3-incubating-SNAPSHOT org.apache.nifi nifi-resources - 0.0.2-incubating + 0.0.3-incubating-SNAPSHOT resources runtime zip @@ -710,7 +710,7 @@ org.apache.nifi nifi-docs - 0.0.2-incubating + 0.0.3-incubating-SNAPSHOT resources runtime zip @@ -718,116 +718,116 @@ org.apache.nifi nifi-framework-nar - 0.0.2-incubating + 0.0.3-incubating-SNAPSHOT nar org.apache.nifi nifi-provenance-repository-nar - 0.0.2-incubating + 0.0.3-incubating-SNAPSHOT nar org.apache.nifi nifi-standard-services-api-nar - 0.0.2-incubating + 0.0.3-incubating-SNAPSHOT nar org.apache.nifi nifi-ssl-context-service-nar - 0.0.2-incubating + 0.0.3-incubating-SNAPSHOT nar org.apache.nifi nifi-distributed-cache-services-nar - 0.0.2-incubating + 0.0.3-incubating-SNAPSHOT nar org.apache.nifi nifi-standard-nar - 0.0.2-incubating + 0.0.3-incubating-SNAPSHOT nar org.apache.nifi nifi-jetty-bundle - 0.0.2-incubating + 0.0.3-incubating-SNAPSHOT nar org.apache.nifi nifi-update-attribute-nar - 0.0.2-incubating + 0.0.3-incubating-SNAPSHOT nar org.apache.nifi nifi-hadoop-libraries-nar - 0.0.2-incubating + 0.0.3-incubating-SNAPSHOT nar org.apache.nifi nifi-hadoop-nar - 0.0.2-incubating + 0.0.3-incubating-SNAPSHOT nar org.apache.nifi nifi-kite-nar - 0.0.2-incubating + 0.0.3-incubating-SNAPSHOT nar org.apache.nifi nifi-kafka-nar - 0.0.2-incubating + 0.0.3-incubating-SNAPSHOT nar org.apache.nifi nifi-http-context-map-nar - 0.0.2-incubating + 0.0.3-incubating-SNAPSHOT nar org.apache.nifi nifi-properties - 0.0.2-incubating + 0.0.3-incubating-SNAPSHOT org.apache.nifi nifi-security-utils - 0.0.2-incubating + 0.0.3-incubating-SNAPSHOT org.apache.nifi nifi-logging-utils - 0.0.2-incubating + 0.0.3-incubating-SNAPSHOT org.apache.nifi nifi-nar-utils - 0.0.2-incubating + 0.0.3-incubating-SNAPSHOT org.apache.nifi nifi-processor-utils - 0.0.2-incubating + 0.0.3-incubating-SNAPSHOT org.apache.nifi nifi-mock - 0.0.2-incubating + 0.0.3-incubating-SNAPSHOT test org.apache.nifi nifi-write-ahead-log - 0.0.2-incubating + 0.0.3-incubating-SNAPSHOT com.jayway.jsonpath From fcc0078827efbdffc0929019e14103c2cfd1ee8f Mon Sep 17 00:00:00 2001 From: Jenn Barnabee Date: Fri, 13 Mar 2015 07:24:04 -0400 Subject: [PATCH 16/26] NIFI-407 Added the property nifi.bored.yield.duration and its description to the Admin Guide. --- nifi/nifi-docs/src/main/asciidoc/administration-guide.adoc | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/nifi/nifi-docs/src/main/asciidoc/administration-guide.adoc b/nifi/nifi-docs/src/main/asciidoc/administration-guide.adoc index 4c4df308c0..b2ad54f662 100644 --- a/nifi/nifi-docs/src/main/asciidoc/administration-guide.adoc +++ b/nifi/nifi-docs/src/main/asciidoc/administration-guide.adoc @@ -301,7 +301,7 @@ The first section of the _nifi.properties_ file is for the Core Properties. Thes |nifi.flowcontroller.autoResumeState|Indicates whether -upon restart- the components on the NiFi graph should return to their last state. The default value is _true_. |nifi.flowcontroller.graceful.shutdown.period|Indicates the shutdown period. The default value is 10 sec. |nifi.flowservice.writedelay.interval|When many changes are made to the flow.xml, this property specifies how long to wait before writing out the changes, so as to batch the changes into a single write. The default value is 500 ms. -|nifi.administrative.yield.duration|If a component allows an unexpected Exception to escape, it is considered a bug. As a result, the framework will pause (or administratively yield) the component for this amount of time. This is done so that the component does not use up massive amounts of system resources, since it is known to have problems in the existing state. The default value is 30 sec. +|nifi.administrative.yield.duration|If a component allows an unexpected exception to escape, it is considered a bug. As a result, the framework will pause (or administratively yield) the component for this amount of time. This is done so that the component does not use up massive amounts of system resources, since it is known to have problems in the existing state. The default value is 30 sec. |nifi.authority.provider.configuration.file*|This is the location of the file that specifies how user access is authenticated. The default value is ./conf/authority-providers.xml. |nifi.reporting.task.configuration.file*|This is the location of the Reporting Tasks file. The default value is ./conf/reporting-tasks.xml. |nifi.controller.service.configuration.file*|This is the location of the Controller Services file. The default value is ./conf/controller-services.xml. @@ -311,6 +311,7 @@ The first section of the _nifi.properties_ file is for the Core Properties. Thes |nifi.nar.library.directory|The location of the nar library. The default value is ./lib and probably should be left as is. |nifi.nar.working.directory|The location of the nar working directory. The default value is ./work/nar and probably should be left as is. |nifi.documentation.working.directory|The documentation working directory. The default value is ./work/docs/components and probably should be left as is. +|nifi.bored.yield.duration|When a processor has no work to do (i.e., is "bored"), this is the amount of time it will wait before checking to see if it has data to work on. This way, it does not use up CPU resources by checking for new work too often. When setting this property, be aware that it could add extra latency for processors that constantly have work to do, as they will always wait this amount of time before checking for more work. The default value is 10 millis. |==== *H2 Settings* + From cc890e291c8f05e7369be45b8299af8764414f98 Mon Sep 17 00:00:00 2001 From: Jenn Barnabee Date: Fri, 13 Mar 2015 08:58:23 -0400 Subject: [PATCH 17/26] NIFI-407a Adds the property nifi.bored.yield.duration with a better description. --- nifi/nifi-docs/src/main/asciidoc/administration-guide.adoc | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/nifi/nifi-docs/src/main/asciidoc/administration-guide.adoc b/nifi/nifi-docs/src/main/asciidoc/administration-guide.adoc index b2ad54f662..96dd423619 100644 --- a/nifi/nifi-docs/src/main/asciidoc/administration-guide.adoc +++ b/nifi/nifi-docs/src/main/asciidoc/administration-guide.adoc @@ -305,13 +305,13 @@ The first section of the _nifi.properties_ file is for the Core Properties. Thes |nifi.authority.provider.configuration.file*|This is the location of the file that specifies how user access is authenticated. The default value is ./conf/authority-providers.xml. |nifi.reporting.task.configuration.file*|This is the location of the Reporting Tasks file. The default value is ./conf/reporting-tasks.xml. |nifi.controller.service.configuration.file*|This is the location of the Controller Services file. The default value is ./conf/controller-services.xml. -|nifi.templates.directory*|This is the location of the directory where flow templates are saved. The default value is ./conf/templates. +|nifi.templates.directory*|This is the location of the directory where flow templates are saved. The default value is ./conf/templates.l |nifi.ui.banner.text|This is banner text that may be configured to display at the top of the User Interface. It is blank by default. |nifi.ui.autorefresh.interval|The interval at which the User Interface auto-refreshes. The default value is 30 sec. |nifi.nar.library.directory|The location of the nar library. The default value is ./lib and probably should be left as is. |nifi.nar.working.directory|The location of the nar working directory. The default value is ./work/nar and probably should be left as is. |nifi.documentation.working.directory|The documentation working directory. The default value is ./work/docs/components and probably should be left as is. -|nifi.bored.yield.duration|When a processor has no work to do (i.e., is "bored"), this is the amount of time it will wait before checking to see if it has data to work on. This way, it does not use up CPU resources by checking for new work too often. When setting this property, be aware that it could add extra latency for processors that constantly have work to do, as they will always wait this amount of time before checking for more work. The default value is 10 millis. +|nifi.bored.yield.duration|When a component has no work to do (i.e., is "bored"), this is the amount of time it will wait before checking to see if it has new data to work on. This way, it does not use up CPU resources by checking for new work too often. When setting this property, be aware that it could add extra latency for components that do not constantly have work to do, as once they go into this "bored" state, they will wait this amount of time before checking for more work. The default value is 10 millis. |==== *H2 Settings* + From cf49dad2712a92cf8c3e53e0d3070b33482cc97b Mon Sep 17 00:00:00 2001 From: joewitt Date: Sat, 14 Mar 2015 22:36:14 -0400 Subject: [PATCH 18/26] NIFI-410 removed unecessary NOTICE for JsonPath PathCompiler --- nifi/NOTICE | 5 ----- 1 file changed, 5 deletions(-) diff --git a/nifi/NOTICE b/nifi/NOTICE index 43af4d103f..9686fbacf0 100644 --- a/nifi/NOTICE +++ b/nifi/NOTICE @@ -4,8 +4,3 @@ Copyright 2014-2015 The Apache Software Foundation This product includes software developed at The Apache Software Foundation (http://www.apache.org/). -The source release of this product bundles a modified -version of PathCompiler from https://github.com/jayway/JsonPath -The following notice information applies - Copyright 2011 Kalle Stenflo, Jochen Berger - From 2de2134c64ff59ea49abe5397a167a8ae5e00885 Mon Sep 17 00:00:00 2001 From: joewitt Date: Sat, 14 Mar 2015 23:53:23 -0400 Subject: [PATCH 19/26] NIFI-410 adding license and notice for nifi-resources to ensure binary artifact has them --- .../nifi-framework/nifi-resources/LICENSE | 202 ++++++++++++++++++ .../nifi-framework/nifi-resources/NOTICE | 5 + .../src/main/assembly/dependencies.xml | 16 ++ 3 files changed, 223 insertions(+) create mode 100644 nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/LICENSE create mode 100644 nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/NOTICE diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/LICENSE b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/LICENSE new file mode 100644 index 0000000000..d645695673 --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/LICENSE @@ -0,0 +1,202 @@ + + Apache License + Version 2.0, January 2004 + http://www.apache.org/licenses/ + + TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION + + 1. Definitions. + + "License" shall mean the terms and conditions for use, reproduction, + and distribution as defined by Sections 1 through 9 of this document. + + "Licensor" shall mean the copyright owner or entity authorized by + the copyright owner that is granting the License. + + "Legal Entity" shall mean the union of the acting entity and all + other entities that control, are controlled by, or are under common + control with that entity. For the purposes of this definition, + "control" means (i) the power, direct or indirect, to cause the + direction or management of such entity, whether by contract or + otherwise, or (ii) ownership of fifty percent (50%) or more of the + outstanding shares, or (iii) beneficial ownership of such entity. + + "You" (or "Your") shall mean an individual or Legal Entity + exercising permissions granted by this License. + + "Source" form shall mean the preferred form for making modifications, + including but not limited to software source code, documentation + source, and configuration files. + + "Object" form shall mean any form resulting from mechanical + transformation or translation of a Source form, including but + not limited to compiled object code, generated documentation, + and conversions to other media types. + + "Work" shall mean the work of authorship, whether in Source or + Object form, made available under the License, as indicated by a + copyright notice that is included in or attached to the work + (an example is provided in the Appendix below). + + "Derivative Works" shall mean any work, whether in Source or Object + form, that is based on (or derived from) the Work and for which the + editorial revisions, annotations, elaborations, or other modifications + represent, as a whole, an original work of authorship. For the purposes + of this License, Derivative Works shall not include works that remain + separable from, or merely link (or bind by name) to the interfaces of, + the Work and Derivative Works thereof. + + "Contribution" shall mean any work of authorship, including + the original version of the Work and any modifications or additions + to that Work or Derivative Works thereof, that is intentionally + submitted to Licensor for inclusion in the Work by the copyright owner + or by an individual or Legal Entity authorized to submit on behalf of + the copyright owner. For the purposes of this definition, "submitted" + means any form of electronic, verbal, or written communication sent + to the Licensor or its representatives, including but not limited to + communication on electronic mailing lists, source code control systems, + and issue tracking systems that are managed by, or on behalf of, the + Licensor for the purpose of discussing and improving the Work, but + excluding communication that is conspicuously marked or otherwise + designated in writing by the copyright owner as "Not a Contribution." + + "Contributor" shall mean Licensor and any individual or Legal Entity + on behalf of whom a Contribution has been received by Licensor and + subsequently incorporated within the Work. + + 2. Grant of Copyright License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + copyright license to reproduce, prepare Derivative Works of, + publicly display, publicly perform, sublicense, and distribute the + Work and such Derivative Works in Source or Object form. + + 3. Grant of Patent License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + (except as stated in this section) patent license to make, have made, + use, offer to sell, sell, import, and otherwise transfer the Work, + where such license applies only to those patent claims licensable + by such Contributor that are necessarily infringed by their + Contribution(s) alone or by combination of their Contribution(s) + with the Work to which such Contribution(s) was submitted. If You + institute patent litigation against any entity (including a + cross-claim or counterclaim in a lawsuit) alleging that the Work + or a Contribution incorporated within the Work constitutes direct + or contributory patent infringement, then any patent licenses + granted to You under this License for that Work shall terminate + as of the date such litigation is filed. + + 4. Redistribution. You may reproduce and distribute copies of the + Work or Derivative Works thereof in any medium, with or without + modifications, and in Source or Object form, provided that You + meet the following conditions: + + (a) You must give any other recipients of the Work or + Derivative Works a copy of this License; and + + (b) You must cause any modified files to carry prominent notices + stating that You changed the files; and + + (c) You must retain, in the Source form of any Derivative Works + that You distribute, all copyright, patent, trademark, and + attribution notices from the Source form of the Work, + excluding those notices that do not pertain to any part of + the Derivative Works; and + + (d) If the Work includes a "NOTICE" text file as part of its + distribution, then any Derivative Works that You distribute must + include a readable copy of the attribution notices contained + within such NOTICE file, excluding those notices that do not + pertain to any part of the Derivative Works, in at least one + of the following places: within a NOTICE text file distributed + as part of the Derivative Works; within the Source form or + documentation, if provided along with the Derivative Works; or, + within a display generated by the Derivative Works, if and + wherever such third-party notices normally appear. The contents + of the NOTICE file are for informational purposes only and + do not modify the License. You may add Your own attribution + notices within Derivative Works that You distribute, alongside + or as an addendum to the NOTICE text from the Work, provided + that such additional attribution notices cannot be construed + as modifying the License. + + You may add Your own copyright statement to Your modifications and + may provide additional or different license terms and conditions + for use, reproduction, or distribution of Your modifications, or + for any such Derivative Works as a whole, provided Your use, + reproduction, and distribution of the Work otherwise complies with + the conditions stated in this License. + + 5. Submission of Contributions. Unless You explicitly state otherwise, + any Contribution intentionally submitted for inclusion in the Work + by You to the Licensor shall be under the terms and conditions of + this License, without any additional terms or conditions. + Notwithstanding the above, nothing herein shall supersede or modify + the terms of any separate license agreement you may have executed + with Licensor regarding such Contributions. + + 6. Trademarks. This License does not grant permission to use the trade + names, trademarks, service marks, or product names of the Licensor, + except as required for reasonable and customary use in describing the + origin of the Work and reproducing the content of the NOTICE file. + + 7. Disclaimer of Warranty. Unless required by applicable law or + agreed to in writing, Licensor provides the Work (and each + Contributor provides its Contributions) on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + implied, including, without limitation, any warranties or conditions + of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A + PARTICULAR PURPOSE. You are solely responsible for determining the + appropriateness of using or redistributing the Work and assume any + risks associated with Your exercise of permissions under this License. + + 8. Limitation of Liability. In no event and under no legal theory, + whether in tort (including negligence), contract, or otherwise, + unless required by applicable law (such as deliberate and grossly + negligent acts) or agreed to in writing, shall any Contributor be + liable to You for damages, including any direct, indirect, special, + incidental, or consequential damages of any character arising as a + result of this License or out of the use or inability to use the + Work (including but not limited to damages for loss of goodwill, + work stoppage, computer failure or malfunction, or any and all + other commercial damages or losses), even if such Contributor + has been advised of the possibility of such damages. + + 9. Accepting Warranty or Additional Liability. While redistributing + the Work or Derivative Works thereof, You may choose to offer, + and charge a fee for, acceptance of support, warranty, indemnity, + or other liability obligations and/or rights consistent with this + License. However, in accepting such obligations, You may act only + on Your own behalf and on Your sole responsibility, not on behalf + of any other Contributor, and only if You agree to indemnify, + defend, and hold each Contributor harmless for any liability + incurred by, or claims asserted against, such Contributor by reason + of your accepting any such warranty or additional liability. + + END OF TERMS AND CONDITIONS + + APPENDIX: How to apply the Apache License to your work. + + To apply the Apache License to your work, attach the following + boilerplate notice, with the fields enclosed by brackets "[]" + replaced with your own identifying information. (Don't include + the brackets!) The text should be enclosed in the appropriate + comment syntax for the file format. We also recommend that a + file or class name and description of purpose be included on the + same "printed page" as the copyright notice for easier + identification within third-party archives. + + Copyright [yyyy] [name of copyright owner] + + Licensed 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. diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/NOTICE b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/NOTICE new file mode 100644 index 0000000000..f531eab2d5 --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/NOTICE @@ -0,0 +1,5 @@ +nifi-resources +Copyright 2014-2015 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/assembly/dependencies.xml b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/assembly/dependencies.xml index 12636822fd..2cc37cb2f4 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/assembly/dependencies.xml +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/assembly/dependencies.xml @@ -33,4 +33,20 @@ 0750 + + + ./LICENSE + ./ + LICENSE + 0644 + true + + + ./NOTICE + ./ + NOTICE + 0644 + true + + From 761e64a410a837b4fd756fd730f07621fcea8b37 Mon Sep 17 00:00:00 2001 From: joewitt Date: Mon, 16 Mar 2015 22:48:14 -0400 Subject: [PATCH 20/26] NIFI-410 pushed RAT exclusions down to the relevant modules and stopped doing broad test resource exclusion - is now specific --- .../.gitignore | 0 .../pom.xml | 0 .../ClusterManagerProtocolSender.java | 0 .../cluster/protocol/ConnectionRequest.java | 0 .../cluster/protocol/ConnectionResponse.java | 0 .../nifi/cluster/protocol/Heartbeat.java | 0 .../nifi/cluster/protocol/NodeBulletins.java | 0 .../nifi/cluster/protocol/NodeIdentifier.java | 0 .../cluster/protocol/NodeProtocolSender.java | 0 .../cluster/protocol/ProtocolContext.java | 0 .../cluster/protocol/ProtocolException.java | 0 .../cluster/protocol/ProtocolHandler.java | 0 .../cluster/protocol/ProtocolListener.java | 0 .../protocol/ProtocolMessageMarshaller.java | 0 .../protocol/ProtocolMessageUnmarshaller.java | 0 .../cluster/protocol/StandardDataFlow.java | 0 .../UnknownServiceAddressException.java | 0 .../ClusterManagerProtocolSenderImpl.java | 0 .../ClusterManagerProtocolSenderListener.java | 0 .../impl/ClusterServiceDiscovery.java | 0 .../protocol/impl/ClusterServiceLocator.java | 0 .../impl/ClusterServicesBroadcaster.java | 0 .../protocol/impl/CopyingInputStream.java | 0 .../impl/MulticastProtocolListener.java | 0 .../protocol/impl/NodeProtocolSenderImpl.java | 0 .../impl/NodeProtocolSenderListener.java | 0 .../protocol/impl/SocketProtocolListener.java | 0 .../protocol/jaxb/JaxbProtocolContext.java | 0 .../message/AdaptedConnectionRequest.java | 0 .../message/AdaptedConnectionResponse.java | 0 .../protocol/jaxb/message/AdaptedCounter.java | 0 .../jaxb/message/AdaptedDataFlow.java | 0 .../jaxb/message/AdaptedHeartbeat.java | 0 .../jaxb/message/AdaptedNodeBulletins.java | 0 .../jaxb/message/AdaptedNodeIdentifier.java | 0 .../message/ConnectionRequestAdapter.java | 0 .../message/ConnectionResponseAdapter.java | 0 .../jaxb/message/DataFlowAdapter.java | 0 .../jaxb/message/HeartbeatAdapter.java | 0 .../jaxb/message/JaxbProtocolUtils.java | 0 .../jaxb/message/NodeBulletinsAdapter.java | 0 .../jaxb/message/NodeIdentifierAdapter.java | 0 .../protocol/jaxb/message/ObjectFactory.java | 0 .../message/ConnectionRequestMessage.java | 0 .../message/ConnectionResponseMessage.java | 0 .../ControllerStartupFailureMessage.java | 0 .../protocol/message/DisconnectMessage.java | 0 .../protocol/message/ExceptionMessage.java | 0 .../protocol/message/FlowRequestMessage.java | 0 .../protocol/message/FlowResponseMessage.java | 0 .../protocol/message/HeartbeatMessage.java | 0 .../message/MulticastProtocolMessage.java | 0 .../message/NodeBulletinsMessage.java | 0 .../cluster/protocol/message/PingMessage.java | 0 .../message/PrimaryRoleAssignmentMessage.java | 0 .../protocol/message/ProtocolMessage.java | 0 .../message/ReconnectionFailureMessage.java | 0 .../message/ReconnectionRequestMessage.java | 0 .../message/ReconnectionResponseMessage.java | 0 .../message/ServiceBroadcastMessage.java | 0 .../MulticastConfigurationFactoryBean.java | 0 .../ServerSocketConfigurationFactoryBean.java | 0 .../SocketConfigurationFactoryBean.java | 0 .../nifi-cluster-protocol-context.xml | 0 .../ClusterManagerProtocolSenderImplTest.java | 0 .../impl/ClusterServiceDiscoveryTest.java | 0 .../impl/ClusterServiceLocatorTest.java | 0 .../impl/ClusterServicesBroadcasterTest.java | 0 .../impl/MulticastProtocolListenerTest.java | 0 .../impl/NodeProtocolSenderImplTest.java | 0 .../testutils/DelayedProtocolHandler.java | 0 .../testutils/ReflexiveProtocolHandler.java | 0 .../.gitignore | 0 .../pom.xml | 0 .../nifi/cluster/context/ClusterContext.java | 0 .../cluster/context/ClusterContextImpl.java | 0 .../context/ClusterContextThreadLocal.java | 0 .../ClusterAwareOptimisticLockingManager.java | 0 .../.gitignore | 0 .../pom.xml | 18 +++- .../cluster/client/MulticastTestClient.java | 0 .../org/apache/nifi/cluster/event/Event.java | 0 .../nifi/cluster/event/EventManager.java | 0 .../cluster/event/impl/EventManagerImpl.java | 0 .../cluster/firewall/ClusterNodeFirewall.java | 0 .../impl/FileBasedClusterNodeFirewall.java | 0 .../nifi/cluster/flow/ClusterDataFlow.java | 0 .../nifi/cluster/flow/DaoException.java | 0 .../apache/nifi/cluster/flow/DataFlowDao.java | 0 .../flow/DataFlowManagementService.java | 0 .../nifi/cluster/flow/PersistedFlowState.java | 0 .../nifi/cluster/flow/StaleFlowException.java | 0 .../cluster/flow/impl/DataFlowDaoImpl.java | 0 .../impl/DataFlowManagementServiceImpl.java | 0 .../nifi/cluster/manager/ClusterManager.java | 0 .../cluster/manager/HttpClusterManager.java | 0 .../manager/HttpRequestReplicator.java | 0 .../cluster/manager/HttpResponseMapper.java | 0 .../nifi/cluster/manager/NodeResponse.java | 0 .../exception/BlockedByFirewallException.java | 0 .../manager/exception/ClusterException.java | 0 ...ConnectingNodeMutableRequestException.java | 0 ...sconnectedNodeMutableRequestException.java | 0 .../IllegalClusterStateException.java | 0 .../IllegalNodeDeletionException.java | 0 .../IllegalNodeDisconnectionException.java | 0 .../IllegalNodeReconnectionException.java | 0 .../IneligiblePrimaryNodeException.java | 0 .../exception/MutableRequestException.java | 0 .../exception/NoConnectedNodesException.java | 0 .../NoResponseFromNodesException.java | 0 .../exception/NodeDisconnectionException.java | 0 .../exception/NodeReconnectionException.java | 0 .../PrimaryRoleAssignmentException.java | 0 .../SafeModeMutableRequestException.java | 0 .../exception/UnknownNodeException.java | 0 .../exception/UriConstructionException.java | 0 .../manager/impl/ClusteredEventAccess.java | 0 .../impl/ClusteredReportingContext.java | 0 .../impl/HttpRequestReplicatorImpl.java | 0 .../manager/impl/HttpResponseMapperImpl.java | 0 .../manager/impl/WebClusterManager.java | 0 .../org/apache/nifi/cluster/node/Node.java | 0 ...agerProtocolServiceLocatorFactoryBean.java | 0 ...leBasedClusterNodeFirewallFactoryBean.java | 0 .../spring/WebClusterManagerFactoryBean.java | 0 .../reporting/ClusteredReportingTaskNode.java | 0 .../nifi-cluster-manager-context.xml | 0 .../event/impl/EventManagerImplTest.java | 0 .../FileBasedClusterNodeFirewallTest.java | 0 .../DataFlowManagementServiceImplTest.java | 0 .../impl/HttpRequestReplicatorImplTest.java | 0 .../impl/HttpResponseMapperImplTest.java | 0 .../manager/impl/TestWebClusterManager.java | 0 .../manager/testutils/HttpRequest.java | 0 .../manager/testutils/HttpResponse.java | 0 .../manager/testutils/HttpResponseAction.java | 0 .../cluster/manager/testutils/HttpServer.java | 0 .../ClusterManagerProtocolSenderImplTest.java | 0 .../impl/ClusterServiceLocatorTest.java | 0 .../impl/ClusterServicesBroadcasterTest.java | 0 .../impl/MulticastProtocolListenerTest.java | 0 .../impl/NodeProtocolSenderImplTest.java | 0 .../impl/SocketProtocolListenerTest.java | 0 .../testutils/DelayedProtocolHandler.java | 0 .../testutils/ReflexiveProtocolHandler.java | 0 .../src/test/resources/logback-test.xml | 0 .../nifi/cluster/firewall/impl/empty.txt | 0 .../apache/nifi/cluster/firewall/impl/ips.txt | 0 .../nifi-framework-core/pom.xml | 19 +++- .../nifi-web/nifi-web-ui/pom.xml | 13 +++ .../nifi-framework/pom.xml | 6 +- .../nifi-standard-processors/pom.xml | 86 ++++++++++++++++++- nifi/pom.xml | 36 ++++---- 154 files changed, 149 insertions(+), 29 deletions(-) rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-cluster-protocol => nifi-framework-cluster-protocol}/.gitignore (100%) rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-cluster-protocol => nifi-framework-cluster-protocol}/pom.xml (100%) rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-cluster-protocol => nifi-framework-cluster-protocol}/src/main/java/org/apache/nifi/cluster/protocol/ClusterManagerProtocolSender.java (100%) rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-cluster-protocol => nifi-framework-cluster-protocol}/src/main/java/org/apache/nifi/cluster/protocol/ConnectionRequest.java (100%) rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-cluster-protocol => nifi-framework-cluster-protocol}/src/main/java/org/apache/nifi/cluster/protocol/ConnectionResponse.java (100%) rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-cluster-protocol => nifi-framework-cluster-protocol}/src/main/java/org/apache/nifi/cluster/protocol/Heartbeat.java (100%) rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-cluster-protocol => nifi-framework-cluster-protocol}/src/main/java/org/apache/nifi/cluster/protocol/NodeBulletins.java (100%) rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-cluster-protocol => nifi-framework-cluster-protocol}/src/main/java/org/apache/nifi/cluster/protocol/NodeIdentifier.java (100%) rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-cluster-protocol => nifi-framework-cluster-protocol}/src/main/java/org/apache/nifi/cluster/protocol/NodeProtocolSender.java (100%) rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-cluster-protocol => nifi-framework-cluster-protocol}/src/main/java/org/apache/nifi/cluster/protocol/ProtocolContext.java (100%) rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-cluster-protocol => nifi-framework-cluster-protocol}/src/main/java/org/apache/nifi/cluster/protocol/ProtocolException.java (100%) rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-cluster-protocol => nifi-framework-cluster-protocol}/src/main/java/org/apache/nifi/cluster/protocol/ProtocolHandler.java (100%) rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-cluster-protocol => nifi-framework-cluster-protocol}/src/main/java/org/apache/nifi/cluster/protocol/ProtocolListener.java (100%) rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-cluster-protocol => nifi-framework-cluster-protocol}/src/main/java/org/apache/nifi/cluster/protocol/ProtocolMessageMarshaller.java (100%) rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-cluster-protocol => nifi-framework-cluster-protocol}/src/main/java/org/apache/nifi/cluster/protocol/ProtocolMessageUnmarshaller.java (100%) rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-cluster-protocol => nifi-framework-cluster-protocol}/src/main/java/org/apache/nifi/cluster/protocol/StandardDataFlow.java (100%) rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-cluster-protocol => nifi-framework-cluster-protocol}/src/main/java/org/apache/nifi/cluster/protocol/UnknownServiceAddressException.java (100%) rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-cluster-protocol => nifi-framework-cluster-protocol}/src/main/java/org/apache/nifi/cluster/protocol/impl/ClusterManagerProtocolSenderImpl.java (100%) rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-cluster-protocol => nifi-framework-cluster-protocol}/src/main/java/org/apache/nifi/cluster/protocol/impl/ClusterManagerProtocolSenderListener.java (100%) rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-cluster-protocol => nifi-framework-cluster-protocol}/src/main/java/org/apache/nifi/cluster/protocol/impl/ClusterServiceDiscovery.java (100%) rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-cluster-protocol => nifi-framework-cluster-protocol}/src/main/java/org/apache/nifi/cluster/protocol/impl/ClusterServiceLocator.java (100%) rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-cluster-protocol => nifi-framework-cluster-protocol}/src/main/java/org/apache/nifi/cluster/protocol/impl/ClusterServicesBroadcaster.java (100%) rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-cluster-protocol => nifi-framework-cluster-protocol}/src/main/java/org/apache/nifi/cluster/protocol/impl/CopyingInputStream.java (100%) rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-cluster-protocol => nifi-framework-cluster-protocol}/src/main/java/org/apache/nifi/cluster/protocol/impl/MulticastProtocolListener.java (100%) rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-cluster-protocol => nifi-framework-cluster-protocol}/src/main/java/org/apache/nifi/cluster/protocol/impl/NodeProtocolSenderImpl.java (100%) rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-cluster-protocol => nifi-framework-cluster-protocol}/src/main/java/org/apache/nifi/cluster/protocol/impl/NodeProtocolSenderListener.java (100%) rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-cluster-protocol => nifi-framework-cluster-protocol}/src/main/java/org/apache/nifi/cluster/protocol/impl/SocketProtocolListener.java (100%) rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-cluster-protocol => nifi-framework-cluster-protocol}/src/main/java/org/apache/nifi/cluster/protocol/jaxb/JaxbProtocolContext.java (100%) rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-cluster-protocol => nifi-framework-cluster-protocol}/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/AdaptedConnectionRequest.java (100%) rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-cluster-protocol => nifi-framework-cluster-protocol}/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/AdaptedConnectionResponse.java (100%) rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-cluster-protocol => nifi-framework-cluster-protocol}/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/AdaptedCounter.java (100%) rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-cluster-protocol => nifi-framework-cluster-protocol}/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/AdaptedDataFlow.java (100%) rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-cluster-protocol => nifi-framework-cluster-protocol}/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/AdaptedHeartbeat.java (100%) rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-cluster-protocol => nifi-framework-cluster-protocol}/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/AdaptedNodeBulletins.java (100%) rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-cluster-protocol => nifi-framework-cluster-protocol}/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/AdaptedNodeIdentifier.java (100%) rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-cluster-protocol => nifi-framework-cluster-protocol}/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/ConnectionRequestAdapter.java (100%) rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-cluster-protocol => nifi-framework-cluster-protocol}/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/ConnectionResponseAdapter.java (100%) rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-cluster-protocol => nifi-framework-cluster-protocol}/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/DataFlowAdapter.java (100%) rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-cluster-protocol => nifi-framework-cluster-protocol}/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/HeartbeatAdapter.java (100%) rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-cluster-protocol => nifi-framework-cluster-protocol}/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/JaxbProtocolUtils.java (100%) rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-cluster-protocol => nifi-framework-cluster-protocol}/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/NodeBulletinsAdapter.java (100%) rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-cluster-protocol => nifi-framework-cluster-protocol}/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/NodeIdentifierAdapter.java (100%) rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-cluster-protocol => nifi-framework-cluster-protocol}/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/ObjectFactory.java (100%) rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-cluster-protocol => nifi-framework-cluster-protocol}/src/main/java/org/apache/nifi/cluster/protocol/message/ConnectionRequestMessage.java (100%) rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-cluster-protocol => nifi-framework-cluster-protocol}/src/main/java/org/apache/nifi/cluster/protocol/message/ConnectionResponseMessage.java (100%) rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-cluster-protocol => nifi-framework-cluster-protocol}/src/main/java/org/apache/nifi/cluster/protocol/message/ControllerStartupFailureMessage.java (100%) rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-cluster-protocol => nifi-framework-cluster-protocol}/src/main/java/org/apache/nifi/cluster/protocol/message/DisconnectMessage.java (100%) rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-cluster-protocol => nifi-framework-cluster-protocol}/src/main/java/org/apache/nifi/cluster/protocol/message/ExceptionMessage.java (100%) rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-cluster-protocol => nifi-framework-cluster-protocol}/src/main/java/org/apache/nifi/cluster/protocol/message/FlowRequestMessage.java (100%) rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-cluster-protocol => nifi-framework-cluster-protocol}/src/main/java/org/apache/nifi/cluster/protocol/message/FlowResponseMessage.java (100%) rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-cluster-protocol => nifi-framework-cluster-protocol}/src/main/java/org/apache/nifi/cluster/protocol/message/HeartbeatMessage.java (100%) rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-cluster-protocol => nifi-framework-cluster-protocol}/src/main/java/org/apache/nifi/cluster/protocol/message/MulticastProtocolMessage.java (100%) rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-cluster-protocol => nifi-framework-cluster-protocol}/src/main/java/org/apache/nifi/cluster/protocol/message/NodeBulletinsMessage.java (100%) rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-cluster-protocol => nifi-framework-cluster-protocol}/src/main/java/org/apache/nifi/cluster/protocol/message/PingMessage.java (100%) rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-cluster-protocol => nifi-framework-cluster-protocol}/src/main/java/org/apache/nifi/cluster/protocol/message/PrimaryRoleAssignmentMessage.java (100%) rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-cluster-protocol => nifi-framework-cluster-protocol}/src/main/java/org/apache/nifi/cluster/protocol/message/ProtocolMessage.java (100%) rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-cluster-protocol => nifi-framework-cluster-protocol}/src/main/java/org/apache/nifi/cluster/protocol/message/ReconnectionFailureMessage.java (100%) rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-cluster-protocol => nifi-framework-cluster-protocol}/src/main/java/org/apache/nifi/cluster/protocol/message/ReconnectionRequestMessage.java (100%) rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-cluster-protocol => nifi-framework-cluster-protocol}/src/main/java/org/apache/nifi/cluster/protocol/message/ReconnectionResponseMessage.java (100%) rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-cluster-protocol => nifi-framework-cluster-protocol}/src/main/java/org/apache/nifi/cluster/protocol/message/ServiceBroadcastMessage.java (100%) rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-cluster-protocol => nifi-framework-cluster-protocol}/src/main/java/org/apache/nifi/cluster/protocol/spring/MulticastConfigurationFactoryBean.java (100%) rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-cluster-protocol => nifi-framework-cluster-protocol}/src/main/java/org/apache/nifi/cluster/protocol/spring/ServerSocketConfigurationFactoryBean.java (100%) rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-cluster-protocol => nifi-framework-cluster-protocol}/src/main/java/org/apache/nifi/cluster/protocol/spring/SocketConfigurationFactoryBean.java (100%) rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-cluster-protocol => nifi-framework-cluster-protocol}/src/main/resources/nifi-cluster-protocol-context.xml (100%) rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-cluster-protocol => nifi-framework-cluster-protocol}/src/test/java/org/apache/nifi/cluster/protocol/impl/ClusterManagerProtocolSenderImplTest.java (100%) rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-cluster-protocol => nifi-framework-cluster-protocol}/src/test/java/org/apache/nifi/cluster/protocol/impl/ClusterServiceDiscoveryTest.java (100%) rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-cluster-protocol => nifi-framework-cluster-protocol}/src/test/java/org/apache/nifi/cluster/protocol/impl/ClusterServiceLocatorTest.java (100%) rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-cluster-protocol => nifi-framework-cluster-protocol}/src/test/java/org/apache/nifi/cluster/protocol/impl/ClusterServicesBroadcasterTest.java (100%) rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-cluster-protocol => nifi-framework-cluster-protocol}/src/test/java/org/apache/nifi/cluster/protocol/impl/MulticastProtocolListenerTest.java (100%) rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-cluster-protocol => nifi-framework-cluster-protocol}/src/test/java/org/apache/nifi/cluster/protocol/impl/NodeProtocolSenderImplTest.java (100%) rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-cluster-protocol => nifi-framework-cluster-protocol}/src/test/java/org/apache/nifi/cluster/protocol/impl/testutils/DelayedProtocolHandler.java (100%) rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-cluster-protocol => nifi-framework-cluster-protocol}/src/test/java/org/apache/nifi/cluster/protocol/impl/testutils/ReflexiveProtocolHandler.java (100%) rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-cluster-web => nifi-framework-cluster-web}/.gitignore (100%) rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-cluster-web => nifi-framework-cluster-web}/pom.xml (100%) rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-cluster-web => nifi-framework-cluster-web}/src/main/java/org/apache/nifi/cluster/context/ClusterContext.java (100%) rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-cluster-web => nifi-framework-cluster-web}/src/main/java/org/apache/nifi/cluster/context/ClusterContextImpl.java (100%) rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-cluster-web => nifi-framework-cluster-web}/src/main/java/org/apache/nifi/cluster/context/ClusterContextThreadLocal.java (100%) rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-cluster-web => nifi-framework-cluster-web}/src/main/java/org/apache/nifi/web/ClusterAwareOptimisticLockingManager.java (100%) rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-cluster => nifi-framework-cluster}/.gitignore (100%) rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-cluster => nifi-framework-cluster}/pom.xml (87%) rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-cluster => nifi-framework-cluster}/src/main/java/org/apache/nifi/cluster/client/MulticastTestClient.java (100%) rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-cluster => nifi-framework-cluster}/src/main/java/org/apache/nifi/cluster/event/Event.java (100%) rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-cluster => nifi-framework-cluster}/src/main/java/org/apache/nifi/cluster/event/EventManager.java (100%) rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-cluster => nifi-framework-cluster}/src/main/java/org/apache/nifi/cluster/event/impl/EventManagerImpl.java (100%) rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-cluster => nifi-framework-cluster}/src/main/java/org/apache/nifi/cluster/firewall/ClusterNodeFirewall.java (100%) rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-cluster => nifi-framework-cluster}/src/main/java/org/apache/nifi/cluster/firewall/impl/FileBasedClusterNodeFirewall.java (100%) rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-cluster => nifi-framework-cluster}/src/main/java/org/apache/nifi/cluster/flow/ClusterDataFlow.java (100%) rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-cluster => nifi-framework-cluster}/src/main/java/org/apache/nifi/cluster/flow/DaoException.java (100%) rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-cluster => nifi-framework-cluster}/src/main/java/org/apache/nifi/cluster/flow/DataFlowDao.java (100%) rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-cluster => nifi-framework-cluster}/src/main/java/org/apache/nifi/cluster/flow/DataFlowManagementService.java (100%) rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-cluster => nifi-framework-cluster}/src/main/java/org/apache/nifi/cluster/flow/PersistedFlowState.java (100%) rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-cluster => nifi-framework-cluster}/src/main/java/org/apache/nifi/cluster/flow/StaleFlowException.java (100%) rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-cluster => nifi-framework-cluster}/src/main/java/org/apache/nifi/cluster/flow/impl/DataFlowDaoImpl.java (100%) rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-cluster => nifi-framework-cluster}/src/main/java/org/apache/nifi/cluster/flow/impl/DataFlowManagementServiceImpl.java (100%) rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-cluster => nifi-framework-cluster}/src/main/java/org/apache/nifi/cluster/manager/ClusterManager.java (100%) rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-cluster => nifi-framework-cluster}/src/main/java/org/apache/nifi/cluster/manager/HttpClusterManager.java (100%) rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-cluster => nifi-framework-cluster}/src/main/java/org/apache/nifi/cluster/manager/HttpRequestReplicator.java (100%) rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-cluster => nifi-framework-cluster}/src/main/java/org/apache/nifi/cluster/manager/HttpResponseMapper.java (100%) rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-cluster => nifi-framework-cluster}/src/main/java/org/apache/nifi/cluster/manager/NodeResponse.java (100%) rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-cluster => nifi-framework-cluster}/src/main/java/org/apache/nifi/cluster/manager/exception/BlockedByFirewallException.java (100%) rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-cluster => nifi-framework-cluster}/src/main/java/org/apache/nifi/cluster/manager/exception/ClusterException.java (100%) rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-cluster => nifi-framework-cluster}/src/main/java/org/apache/nifi/cluster/manager/exception/ConnectingNodeMutableRequestException.java (100%) rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-cluster => nifi-framework-cluster}/src/main/java/org/apache/nifi/cluster/manager/exception/DisconnectedNodeMutableRequestException.java (100%) rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-cluster => nifi-framework-cluster}/src/main/java/org/apache/nifi/cluster/manager/exception/IllegalClusterStateException.java (100%) rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-cluster => nifi-framework-cluster}/src/main/java/org/apache/nifi/cluster/manager/exception/IllegalNodeDeletionException.java (100%) rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-cluster => nifi-framework-cluster}/src/main/java/org/apache/nifi/cluster/manager/exception/IllegalNodeDisconnectionException.java (100%) rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-cluster => nifi-framework-cluster}/src/main/java/org/apache/nifi/cluster/manager/exception/IllegalNodeReconnectionException.java (100%) rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-cluster => nifi-framework-cluster}/src/main/java/org/apache/nifi/cluster/manager/exception/IneligiblePrimaryNodeException.java (100%) rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-cluster => nifi-framework-cluster}/src/main/java/org/apache/nifi/cluster/manager/exception/MutableRequestException.java (100%) rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-cluster => nifi-framework-cluster}/src/main/java/org/apache/nifi/cluster/manager/exception/NoConnectedNodesException.java (100%) rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-cluster => nifi-framework-cluster}/src/main/java/org/apache/nifi/cluster/manager/exception/NoResponseFromNodesException.java (100%) rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-cluster => nifi-framework-cluster}/src/main/java/org/apache/nifi/cluster/manager/exception/NodeDisconnectionException.java (100%) rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-cluster => nifi-framework-cluster}/src/main/java/org/apache/nifi/cluster/manager/exception/NodeReconnectionException.java (100%) rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-cluster => nifi-framework-cluster}/src/main/java/org/apache/nifi/cluster/manager/exception/PrimaryRoleAssignmentException.java (100%) rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-cluster => nifi-framework-cluster}/src/main/java/org/apache/nifi/cluster/manager/exception/SafeModeMutableRequestException.java (100%) rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-cluster => nifi-framework-cluster}/src/main/java/org/apache/nifi/cluster/manager/exception/UnknownNodeException.java (100%) rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-cluster => nifi-framework-cluster}/src/main/java/org/apache/nifi/cluster/manager/exception/UriConstructionException.java (100%) rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-cluster => nifi-framework-cluster}/src/main/java/org/apache/nifi/cluster/manager/impl/ClusteredEventAccess.java (100%) rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-cluster => nifi-framework-cluster}/src/main/java/org/apache/nifi/cluster/manager/impl/ClusteredReportingContext.java (100%) rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-cluster => nifi-framework-cluster}/src/main/java/org/apache/nifi/cluster/manager/impl/HttpRequestReplicatorImpl.java (100%) rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-cluster => nifi-framework-cluster}/src/main/java/org/apache/nifi/cluster/manager/impl/HttpResponseMapperImpl.java (100%) rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-cluster => nifi-framework-cluster}/src/main/java/org/apache/nifi/cluster/manager/impl/WebClusterManager.java (100%) rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-cluster => nifi-framework-cluster}/src/main/java/org/apache/nifi/cluster/node/Node.java (100%) rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-cluster => nifi-framework-cluster}/src/main/java/org/apache/nifi/cluster/spring/ClusterManagerProtocolServiceLocatorFactoryBean.java (100%) rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-cluster => nifi-framework-cluster}/src/main/java/org/apache/nifi/cluster/spring/FileBasedClusterNodeFirewallFactoryBean.java (100%) rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-cluster => nifi-framework-cluster}/src/main/java/org/apache/nifi/cluster/spring/WebClusterManagerFactoryBean.java (100%) rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-cluster => nifi-framework-cluster}/src/main/java/org/apache/nifi/controller/reporting/ClusteredReportingTaskNode.java (100%) rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-cluster => nifi-framework-cluster}/src/main/resources/nifi-cluster-manager-context.xml (100%) rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-cluster => nifi-framework-cluster}/src/test/java/org/apache/nifi/cluster/event/impl/EventManagerImplTest.java (100%) rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-cluster => nifi-framework-cluster}/src/test/java/org/apache/nifi/cluster/firewall/impl/FileBasedClusterNodeFirewallTest.java (100%) rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-cluster => nifi-framework-cluster}/src/test/java/org/apache/nifi/cluster/flow/impl/DataFlowManagementServiceImplTest.java (100%) rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-cluster => nifi-framework-cluster}/src/test/java/org/apache/nifi/cluster/manager/impl/HttpRequestReplicatorImplTest.java (100%) rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-cluster => nifi-framework-cluster}/src/test/java/org/apache/nifi/cluster/manager/impl/HttpResponseMapperImplTest.java (100%) rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-cluster => nifi-framework-cluster}/src/test/java/org/apache/nifi/cluster/manager/impl/TestWebClusterManager.java (100%) rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-cluster => nifi-framework-cluster}/src/test/java/org/apache/nifi/cluster/manager/testutils/HttpRequest.java (100%) rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-cluster => nifi-framework-cluster}/src/test/java/org/apache/nifi/cluster/manager/testutils/HttpResponse.java (100%) rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-cluster => nifi-framework-cluster}/src/test/java/org/apache/nifi/cluster/manager/testutils/HttpResponseAction.java (100%) rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-cluster => nifi-framework-cluster}/src/test/java/org/apache/nifi/cluster/manager/testutils/HttpServer.java (100%) rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-cluster => nifi-framework-cluster}/src/test/java/org/apache/nifi/cluster/protocol/impl/ClusterManagerProtocolSenderImplTest.java (100%) rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-cluster => nifi-framework-cluster}/src/test/java/org/apache/nifi/cluster/protocol/impl/ClusterServiceLocatorTest.java (100%) rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-cluster => nifi-framework-cluster}/src/test/java/org/apache/nifi/cluster/protocol/impl/ClusterServicesBroadcasterTest.java (100%) rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-cluster => nifi-framework-cluster}/src/test/java/org/apache/nifi/cluster/protocol/impl/MulticastProtocolListenerTest.java (100%) rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-cluster => nifi-framework-cluster}/src/test/java/org/apache/nifi/cluster/protocol/impl/NodeProtocolSenderImplTest.java (100%) rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-cluster => nifi-framework-cluster}/src/test/java/org/apache/nifi/cluster/protocol/impl/SocketProtocolListenerTest.java (100%) rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-cluster => nifi-framework-cluster}/src/test/java/org/apache/nifi/cluster/protocol/testutils/DelayedProtocolHandler.java (100%) rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-cluster => nifi-framework-cluster}/src/test/java/org/apache/nifi/cluster/protocol/testutils/ReflexiveProtocolHandler.java (100%) rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-cluster => nifi-framework-cluster}/src/test/resources/logback-test.xml (100%) rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-cluster => nifi-framework-cluster}/src/test/resources/org/apache/nifi/cluster/firewall/impl/empty.txt (100%) rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-cluster => nifi-framework-cluster}/src/test/resources/org/apache/nifi/cluster/firewall/impl/ips.txt (100%) diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-protocol/.gitignore b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/.gitignore similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-protocol/.gitignore rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/.gitignore diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-protocol/pom.xml b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/pom.xml similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-protocol/pom.xml rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/pom.xml diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/ClusterManagerProtocolSender.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/ClusterManagerProtocolSender.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/ClusterManagerProtocolSender.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/ClusterManagerProtocolSender.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/ConnectionRequest.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/ConnectionRequest.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/ConnectionRequest.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/ConnectionRequest.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/ConnectionResponse.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/ConnectionResponse.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/ConnectionResponse.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/ConnectionResponse.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/Heartbeat.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/Heartbeat.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/Heartbeat.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/Heartbeat.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/NodeBulletins.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/NodeBulletins.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/NodeBulletins.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/NodeBulletins.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/NodeIdentifier.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/NodeIdentifier.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/NodeIdentifier.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/NodeIdentifier.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/NodeProtocolSender.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/NodeProtocolSender.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/NodeProtocolSender.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/NodeProtocolSender.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/ProtocolContext.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/ProtocolContext.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/ProtocolContext.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/ProtocolContext.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/ProtocolException.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/ProtocolException.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/ProtocolException.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/ProtocolException.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/ProtocolHandler.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/ProtocolHandler.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/ProtocolHandler.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/ProtocolHandler.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/ProtocolListener.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/ProtocolListener.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/ProtocolListener.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/ProtocolListener.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/ProtocolMessageMarshaller.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/ProtocolMessageMarshaller.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/ProtocolMessageMarshaller.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/ProtocolMessageMarshaller.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/ProtocolMessageUnmarshaller.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/ProtocolMessageUnmarshaller.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/ProtocolMessageUnmarshaller.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/ProtocolMessageUnmarshaller.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/StandardDataFlow.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/StandardDataFlow.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/StandardDataFlow.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/StandardDataFlow.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/UnknownServiceAddressException.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/UnknownServiceAddressException.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/UnknownServiceAddressException.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/UnknownServiceAddressException.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/ClusterManagerProtocolSenderImpl.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/ClusterManagerProtocolSenderImpl.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/ClusterManagerProtocolSenderImpl.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/ClusterManagerProtocolSenderImpl.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/ClusterManagerProtocolSenderListener.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/ClusterManagerProtocolSenderListener.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/ClusterManagerProtocolSenderListener.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/ClusterManagerProtocolSenderListener.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/ClusterServiceDiscovery.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/ClusterServiceDiscovery.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/ClusterServiceDiscovery.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/ClusterServiceDiscovery.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/ClusterServiceLocator.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/ClusterServiceLocator.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/ClusterServiceLocator.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/ClusterServiceLocator.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/ClusterServicesBroadcaster.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/ClusterServicesBroadcaster.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/ClusterServicesBroadcaster.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/ClusterServicesBroadcaster.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/CopyingInputStream.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/CopyingInputStream.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/CopyingInputStream.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/CopyingInputStream.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/MulticastProtocolListener.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/MulticastProtocolListener.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/MulticastProtocolListener.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/MulticastProtocolListener.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/NodeProtocolSenderImpl.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/NodeProtocolSenderImpl.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/NodeProtocolSenderImpl.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/NodeProtocolSenderImpl.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/NodeProtocolSenderListener.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/NodeProtocolSenderListener.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/NodeProtocolSenderListener.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/NodeProtocolSenderListener.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/SocketProtocolListener.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/SocketProtocolListener.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/SocketProtocolListener.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/SocketProtocolListener.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/JaxbProtocolContext.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/JaxbProtocolContext.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/JaxbProtocolContext.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/JaxbProtocolContext.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/AdaptedConnectionRequest.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/AdaptedConnectionRequest.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/AdaptedConnectionRequest.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/AdaptedConnectionRequest.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/AdaptedConnectionResponse.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/AdaptedConnectionResponse.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/AdaptedConnectionResponse.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/AdaptedConnectionResponse.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/AdaptedCounter.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/AdaptedCounter.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/AdaptedCounter.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/AdaptedCounter.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/AdaptedDataFlow.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/AdaptedDataFlow.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/AdaptedDataFlow.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/AdaptedDataFlow.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/AdaptedHeartbeat.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/AdaptedHeartbeat.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/AdaptedHeartbeat.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/AdaptedHeartbeat.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/AdaptedNodeBulletins.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/AdaptedNodeBulletins.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/AdaptedNodeBulletins.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/AdaptedNodeBulletins.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/AdaptedNodeIdentifier.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/AdaptedNodeIdentifier.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/AdaptedNodeIdentifier.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/AdaptedNodeIdentifier.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/ConnectionRequestAdapter.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/ConnectionRequestAdapter.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/ConnectionRequestAdapter.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/ConnectionRequestAdapter.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/ConnectionResponseAdapter.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/ConnectionResponseAdapter.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/ConnectionResponseAdapter.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/ConnectionResponseAdapter.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/DataFlowAdapter.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/DataFlowAdapter.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/DataFlowAdapter.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/DataFlowAdapter.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/HeartbeatAdapter.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/HeartbeatAdapter.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/HeartbeatAdapter.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/HeartbeatAdapter.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/JaxbProtocolUtils.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/JaxbProtocolUtils.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/JaxbProtocolUtils.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/JaxbProtocolUtils.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/NodeBulletinsAdapter.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/NodeBulletinsAdapter.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/NodeBulletinsAdapter.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/NodeBulletinsAdapter.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/NodeIdentifierAdapter.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/NodeIdentifierAdapter.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/NodeIdentifierAdapter.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/NodeIdentifierAdapter.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/ObjectFactory.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/ObjectFactory.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/ObjectFactory.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/ObjectFactory.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/ConnectionRequestMessage.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/ConnectionRequestMessage.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/ConnectionRequestMessage.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/ConnectionRequestMessage.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/ConnectionResponseMessage.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/ConnectionResponseMessage.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/ConnectionResponseMessage.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/ConnectionResponseMessage.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/ControllerStartupFailureMessage.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/ControllerStartupFailureMessage.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/ControllerStartupFailureMessage.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/ControllerStartupFailureMessage.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/DisconnectMessage.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/DisconnectMessage.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/DisconnectMessage.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/DisconnectMessage.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/ExceptionMessage.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/ExceptionMessage.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/ExceptionMessage.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/ExceptionMessage.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/FlowRequestMessage.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/FlowRequestMessage.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/FlowRequestMessage.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/FlowRequestMessage.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/FlowResponseMessage.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/FlowResponseMessage.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/FlowResponseMessage.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/FlowResponseMessage.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/HeartbeatMessage.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/HeartbeatMessage.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/HeartbeatMessage.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/HeartbeatMessage.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/MulticastProtocolMessage.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/MulticastProtocolMessage.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/MulticastProtocolMessage.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/MulticastProtocolMessage.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/NodeBulletinsMessage.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/NodeBulletinsMessage.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/NodeBulletinsMessage.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/NodeBulletinsMessage.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/PingMessage.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/PingMessage.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/PingMessage.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/PingMessage.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/PrimaryRoleAssignmentMessage.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/PrimaryRoleAssignmentMessage.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/PrimaryRoleAssignmentMessage.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/PrimaryRoleAssignmentMessage.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/ProtocolMessage.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/ProtocolMessage.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/ProtocolMessage.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/ProtocolMessage.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/ReconnectionFailureMessage.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/ReconnectionFailureMessage.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/ReconnectionFailureMessage.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/ReconnectionFailureMessage.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/ReconnectionRequestMessage.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/ReconnectionRequestMessage.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/ReconnectionRequestMessage.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/ReconnectionRequestMessage.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/ReconnectionResponseMessage.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/ReconnectionResponseMessage.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/ReconnectionResponseMessage.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/ReconnectionResponseMessage.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/ServiceBroadcastMessage.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/ServiceBroadcastMessage.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/ServiceBroadcastMessage.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/ServiceBroadcastMessage.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/spring/MulticastConfigurationFactoryBean.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/spring/MulticastConfigurationFactoryBean.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/spring/MulticastConfigurationFactoryBean.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/spring/MulticastConfigurationFactoryBean.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/spring/ServerSocketConfigurationFactoryBean.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/spring/ServerSocketConfigurationFactoryBean.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/spring/ServerSocketConfigurationFactoryBean.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/spring/ServerSocketConfigurationFactoryBean.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/spring/SocketConfigurationFactoryBean.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/spring/SocketConfigurationFactoryBean.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/spring/SocketConfigurationFactoryBean.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/spring/SocketConfigurationFactoryBean.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-protocol/src/main/resources/nifi-cluster-protocol-context.xml b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/resources/nifi-cluster-protocol-context.xml similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-protocol/src/main/resources/nifi-cluster-protocol-context.xml rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/resources/nifi-cluster-protocol-context.xml diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-protocol/src/test/java/org/apache/nifi/cluster/protocol/impl/ClusterManagerProtocolSenderImplTest.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/test/java/org/apache/nifi/cluster/protocol/impl/ClusterManagerProtocolSenderImplTest.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-protocol/src/test/java/org/apache/nifi/cluster/protocol/impl/ClusterManagerProtocolSenderImplTest.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/test/java/org/apache/nifi/cluster/protocol/impl/ClusterManagerProtocolSenderImplTest.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-protocol/src/test/java/org/apache/nifi/cluster/protocol/impl/ClusterServiceDiscoveryTest.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/test/java/org/apache/nifi/cluster/protocol/impl/ClusterServiceDiscoveryTest.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-protocol/src/test/java/org/apache/nifi/cluster/protocol/impl/ClusterServiceDiscoveryTest.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/test/java/org/apache/nifi/cluster/protocol/impl/ClusterServiceDiscoveryTest.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-protocol/src/test/java/org/apache/nifi/cluster/protocol/impl/ClusterServiceLocatorTest.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/test/java/org/apache/nifi/cluster/protocol/impl/ClusterServiceLocatorTest.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-protocol/src/test/java/org/apache/nifi/cluster/protocol/impl/ClusterServiceLocatorTest.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/test/java/org/apache/nifi/cluster/protocol/impl/ClusterServiceLocatorTest.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-protocol/src/test/java/org/apache/nifi/cluster/protocol/impl/ClusterServicesBroadcasterTest.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/test/java/org/apache/nifi/cluster/protocol/impl/ClusterServicesBroadcasterTest.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-protocol/src/test/java/org/apache/nifi/cluster/protocol/impl/ClusterServicesBroadcasterTest.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/test/java/org/apache/nifi/cluster/protocol/impl/ClusterServicesBroadcasterTest.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-protocol/src/test/java/org/apache/nifi/cluster/protocol/impl/MulticastProtocolListenerTest.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/test/java/org/apache/nifi/cluster/protocol/impl/MulticastProtocolListenerTest.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-protocol/src/test/java/org/apache/nifi/cluster/protocol/impl/MulticastProtocolListenerTest.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/test/java/org/apache/nifi/cluster/protocol/impl/MulticastProtocolListenerTest.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-protocol/src/test/java/org/apache/nifi/cluster/protocol/impl/NodeProtocolSenderImplTest.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/test/java/org/apache/nifi/cluster/protocol/impl/NodeProtocolSenderImplTest.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-protocol/src/test/java/org/apache/nifi/cluster/protocol/impl/NodeProtocolSenderImplTest.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/test/java/org/apache/nifi/cluster/protocol/impl/NodeProtocolSenderImplTest.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-protocol/src/test/java/org/apache/nifi/cluster/protocol/impl/testutils/DelayedProtocolHandler.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/test/java/org/apache/nifi/cluster/protocol/impl/testutils/DelayedProtocolHandler.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-protocol/src/test/java/org/apache/nifi/cluster/protocol/impl/testutils/DelayedProtocolHandler.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/test/java/org/apache/nifi/cluster/protocol/impl/testutils/DelayedProtocolHandler.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-protocol/src/test/java/org/apache/nifi/cluster/protocol/impl/testutils/ReflexiveProtocolHandler.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/test/java/org/apache/nifi/cluster/protocol/impl/testutils/ReflexiveProtocolHandler.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-protocol/src/test/java/org/apache/nifi/cluster/protocol/impl/testutils/ReflexiveProtocolHandler.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/test/java/org/apache/nifi/cluster/protocol/impl/testutils/ReflexiveProtocolHandler.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-web/.gitignore b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-web/.gitignore similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-web/.gitignore rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-web/.gitignore diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-web/pom.xml b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-web/pom.xml similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-web/pom.xml rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-web/pom.xml diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-web/src/main/java/org/apache/nifi/cluster/context/ClusterContext.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-web/src/main/java/org/apache/nifi/cluster/context/ClusterContext.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-web/src/main/java/org/apache/nifi/cluster/context/ClusterContext.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-web/src/main/java/org/apache/nifi/cluster/context/ClusterContext.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-web/src/main/java/org/apache/nifi/cluster/context/ClusterContextImpl.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-web/src/main/java/org/apache/nifi/cluster/context/ClusterContextImpl.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-web/src/main/java/org/apache/nifi/cluster/context/ClusterContextImpl.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-web/src/main/java/org/apache/nifi/cluster/context/ClusterContextImpl.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-web/src/main/java/org/apache/nifi/cluster/context/ClusterContextThreadLocal.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-web/src/main/java/org/apache/nifi/cluster/context/ClusterContextThreadLocal.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-web/src/main/java/org/apache/nifi/cluster/context/ClusterContextThreadLocal.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-web/src/main/java/org/apache/nifi/cluster/context/ClusterContextThreadLocal.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-web/src/main/java/org/apache/nifi/web/ClusterAwareOptimisticLockingManager.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-web/src/main/java/org/apache/nifi/web/ClusterAwareOptimisticLockingManager.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster-web/src/main/java/org/apache/nifi/web/ClusterAwareOptimisticLockingManager.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-web/src/main/java/org/apache/nifi/web/ClusterAwareOptimisticLockingManager.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/.gitignore b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/.gitignore similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/.gitignore rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/.gitignore diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/pom.xml b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/pom.xml similarity index 87% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/pom.xml rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/pom.xml index dd24804a36..ef927a31aa 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/pom.xml +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/pom.xml @@ -74,8 +74,8 @@ nifi-site-to-site - org.apache.nifi - nifi-site-to-site-client + org.apache.nifi + nifi-site-to-site-client org.apache.commons @@ -127,4 +127,18 @@ spring-context + + + + org.apache.rat + apache-rat-plugin + + + src/test/resources/org/apache/nifi/cluster/firewall/impl/empty.txt + src/test/resources/org/apache/nifi/cluster/firewall/impl/ips.txt + + + + + diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/main/java/org/apache/nifi/cluster/client/MulticastTestClient.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/client/MulticastTestClient.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/main/java/org/apache/nifi/cluster/client/MulticastTestClient.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/client/MulticastTestClient.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/main/java/org/apache/nifi/cluster/event/Event.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/event/Event.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/main/java/org/apache/nifi/cluster/event/Event.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/event/Event.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/main/java/org/apache/nifi/cluster/event/EventManager.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/event/EventManager.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/main/java/org/apache/nifi/cluster/event/EventManager.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/event/EventManager.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/main/java/org/apache/nifi/cluster/event/impl/EventManagerImpl.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/event/impl/EventManagerImpl.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/main/java/org/apache/nifi/cluster/event/impl/EventManagerImpl.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/event/impl/EventManagerImpl.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/main/java/org/apache/nifi/cluster/firewall/ClusterNodeFirewall.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/firewall/ClusterNodeFirewall.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/main/java/org/apache/nifi/cluster/firewall/ClusterNodeFirewall.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/firewall/ClusterNodeFirewall.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/main/java/org/apache/nifi/cluster/firewall/impl/FileBasedClusterNodeFirewall.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/firewall/impl/FileBasedClusterNodeFirewall.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/main/java/org/apache/nifi/cluster/firewall/impl/FileBasedClusterNodeFirewall.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/firewall/impl/FileBasedClusterNodeFirewall.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/main/java/org/apache/nifi/cluster/flow/ClusterDataFlow.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/flow/ClusterDataFlow.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/main/java/org/apache/nifi/cluster/flow/ClusterDataFlow.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/flow/ClusterDataFlow.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/main/java/org/apache/nifi/cluster/flow/DaoException.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/flow/DaoException.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/main/java/org/apache/nifi/cluster/flow/DaoException.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/flow/DaoException.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/main/java/org/apache/nifi/cluster/flow/DataFlowDao.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/flow/DataFlowDao.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/main/java/org/apache/nifi/cluster/flow/DataFlowDao.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/flow/DataFlowDao.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/main/java/org/apache/nifi/cluster/flow/DataFlowManagementService.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/flow/DataFlowManagementService.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/main/java/org/apache/nifi/cluster/flow/DataFlowManagementService.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/flow/DataFlowManagementService.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/main/java/org/apache/nifi/cluster/flow/PersistedFlowState.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/flow/PersistedFlowState.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/main/java/org/apache/nifi/cluster/flow/PersistedFlowState.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/flow/PersistedFlowState.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/main/java/org/apache/nifi/cluster/flow/StaleFlowException.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/flow/StaleFlowException.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/main/java/org/apache/nifi/cluster/flow/StaleFlowException.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/flow/StaleFlowException.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/main/java/org/apache/nifi/cluster/flow/impl/DataFlowDaoImpl.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/flow/impl/DataFlowDaoImpl.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/main/java/org/apache/nifi/cluster/flow/impl/DataFlowDaoImpl.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/flow/impl/DataFlowDaoImpl.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/main/java/org/apache/nifi/cluster/flow/impl/DataFlowManagementServiceImpl.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/flow/impl/DataFlowManagementServiceImpl.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/main/java/org/apache/nifi/cluster/flow/impl/DataFlowManagementServiceImpl.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/flow/impl/DataFlowManagementServiceImpl.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/main/java/org/apache/nifi/cluster/manager/ClusterManager.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/ClusterManager.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/main/java/org/apache/nifi/cluster/manager/ClusterManager.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/ClusterManager.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/main/java/org/apache/nifi/cluster/manager/HttpClusterManager.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/HttpClusterManager.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/main/java/org/apache/nifi/cluster/manager/HttpClusterManager.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/HttpClusterManager.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/main/java/org/apache/nifi/cluster/manager/HttpRequestReplicator.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/HttpRequestReplicator.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/main/java/org/apache/nifi/cluster/manager/HttpRequestReplicator.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/HttpRequestReplicator.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/main/java/org/apache/nifi/cluster/manager/HttpResponseMapper.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/HttpResponseMapper.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/main/java/org/apache/nifi/cluster/manager/HttpResponseMapper.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/HttpResponseMapper.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/main/java/org/apache/nifi/cluster/manager/NodeResponse.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/NodeResponse.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/main/java/org/apache/nifi/cluster/manager/NodeResponse.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/NodeResponse.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/main/java/org/apache/nifi/cluster/manager/exception/BlockedByFirewallException.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/exception/BlockedByFirewallException.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/main/java/org/apache/nifi/cluster/manager/exception/BlockedByFirewallException.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/exception/BlockedByFirewallException.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/main/java/org/apache/nifi/cluster/manager/exception/ClusterException.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/exception/ClusterException.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/main/java/org/apache/nifi/cluster/manager/exception/ClusterException.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/exception/ClusterException.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/main/java/org/apache/nifi/cluster/manager/exception/ConnectingNodeMutableRequestException.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/exception/ConnectingNodeMutableRequestException.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/main/java/org/apache/nifi/cluster/manager/exception/ConnectingNodeMutableRequestException.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/exception/ConnectingNodeMutableRequestException.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/main/java/org/apache/nifi/cluster/manager/exception/DisconnectedNodeMutableRequestException.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/exception/DisconnectedNodeMutableRequestException.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/main/java/org/apache/nifi/cluster/manager/exception/DisconnectedNodeMutableRequestException.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/exception/DisconnectedNodeMutableRequestException.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/main/java/org/apache/nifi/cluster/manager/exception/IllegalClusterStateException.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/exception/IllegalClusterStateException.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/main/java/org/apache/nifi/cluster/manager/exception/IllegalClusterStateException.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/exception/IllegalClusterStateException.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/main/java/org/apache/nifi/cluster/manager/exception/IllegalNodeDeletionException.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/exception/IllegalNodeDeletionException.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/main/java/org/apache/nifi/cluster/manager/exception/IllegalNodeDeletionException.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/exception/IllegalNodeDeletionException.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/main/java/org/apache/nifi/cluster/manager/exception/IllegalNodeDisconnectionException.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/exception/IllegalNodeDisconnectionException.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/main/java/org/apache/nifi/cluster/manager/exception/IllegalNodeDisconnectionException.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/exception/IllegalNodeDisconnectionException.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/main/java/org/apache/nifi/cluster/manager/exception/IllegalNodeReconnectionException.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/exception/IllegalNodeReconnectionException.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/main/java/org/apache/nifi/cluster/manager/exception/IllegalNodeReconnectionException.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/exception/IllegalNodeReconnectionException.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/main/java/org/apache/nifi/cluster/manager/exception/IneligiblePrimaryNodeException.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/exception/IneligiblePrimaryNodeException.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/main/java/org/apache/nifi/cluster/manager/exception/IneligiblePrimaryNodeException.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/exception/IneligiblePrimaryNodeException.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/main/java/org/apache/nifi/cluster/manager/exception/MutableRequestException.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/exception/MutableRequestException.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/main/java/org/apache/nifi/cluster/manager/exception/MutableRequestException.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/exception/MutableRequestException.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/main/java/org/apache/nifi/cluster/manager/exception/NoConnectedNodesException.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/exception/NoConnectedNodesException.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/main/java/org/apache/nifi/cluster/manager/exception/NoConnectedNodesException.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/exception/NoConnectedNodesException.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/main/java/org/apache/nifi/cluster/manager/exception/NoResponseFromNodesException.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/exception/NoResponseFromNodesException.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/main/java/org/apache/nifi/cluster/manager/exception/NoResponseFromNodesException.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/exception/NoResponseFromNodesException.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/main/java/org/apache/nifi/cluster/manager/exception/NodeDisconnectionException.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/exception/NodeDisconnectionException.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/main/java/org/apache/nifi/cluster/manager/exception/NodeDisconnectionException.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/exception/NodeDisconnectionException.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/main/java/org/apache/nifi/cluster/manager/exception/NodeReconnectionException.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/exception/NodeReconnectionException.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/main/java/org/apache/nifi/cluster/manager/exception/NodeReconnectionException.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/exception/NodeReconnectionException.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/main/java/org/apache/nifi/cluster/manager/exception/PrimaryRoleAssignmentException.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/exception/PrimaryRoleAssignmentException.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/main/java/org/apache/nifi/cluster/manager/exception/PrimaryRoleAssignmentException.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/exception/PrimaryRoleAssignmentException.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/main/java/org/apache/nifi/cluster/manager/exception/SafeModeMutableRequestException.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/exception/SafeModeMutableRequestException.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/main/java/org/apache/nifi/cluster/manager/exception/SafeModeMutableRequestException.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/exception/SafeModeMutableRequestException.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/main/java/org/apache/nifi/cluster/manager/exception/UnknownNodeException.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/exception/UnknownNodeException.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/main/java/org/apache/nifi/cluster/manager/exception/UnknownNodeException.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/exception/UnknownNodeException.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/main/java/org/apache/nifi/cluster/manager/exception/UriConstructionException.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/exception/UriConstructionException.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/main/java/org/apache/nifi/cluster/manager/exception/UriConstructionException.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/exception/UriConstructionException.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/main/java/org/apache/nifi/cluster/manager/impl/ClusteredEventAccess.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/impl/ClusteredEventAccess.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/main/java/org/apache/nifi/cluster/manager/impl/ClusteredEventAccess.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/impl/ClusteredEventAccess.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/main/java/org/apache/nifi/cluster/manager/impl/ClusteredReportingContext.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/impl/ClusteredReportingContext.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/main/java/org/apache/nifi/cluster/manager/impl/ClusteredReportingContext.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/impl/ClusteredReportingContext.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/main/java/org/apache/nifi/cluster/manager/impl/HttpRequestReplicatorImpl.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/impl/HttpRequestReplicatorImpl.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/main/java/org/apache/nifi/cluster/manager/impl/HttpRequestReplicatorImpl.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/impl/HttpRequestReplicatorImpl.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/main/java/org/apache/nifi/cluster/manager/impl/HttpResponseMapperImpl.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/impl/HttpResponseMapperImpl.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/main/java/org/apache/nifi/cluster/manager/impl/HttpResponseMapperImpl.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/impl/HttpResponseMapperImpl.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/main/java/org/apache/nifi/cluster/manager/impl/WebClusterManager.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/impl/WebClusterManager.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/main/java/org/apache/nifi/cluster/manager/impl/WebClusterManager.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/impl/WebClusterManager.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/main/java/org/apache/nifi/cluster/node/Node.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/node/Node.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/main/java/org/apache/nifi/cluster/node/Node.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/node/Node.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/main/java/org/apache/nifi/cluster/spring/ClusterManagerProtocolServiceLocatorFactoryBean.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/spring/ClusterManagerProtocolServiceLocatorFactoryBean.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/main/java/org/apache/nifi/cluster/spring/ClusterManagerProtocolServiceLocatorFactoryBean.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/spring/ClusterManagerProtocolServiceLocatorFactoryBean.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/main/java/org/apache/nifi/cluster/spring/FileBasedClusterNodeFirewallFactoryBean.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/spring/FileBasedClusterNodeFirewallFactoryBean.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/main/java/org/apache/nifi/cluster/spring/FileBasedClusterNodeFirewallFactoryBean.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/spring/FileBasedClusterNodeFirewallFactoryBean.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/main/java/org/apache/nifi/cluster/spring/WebClusterManagerFactoryBean.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/spring/WebClusterManagerFactoryBean.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/main/java/org/apache/nifi/cluster/spring/WebClusterManagerFactoryBean.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/spring/WebClusterManagerFactoryBean.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/main/java/org/apache/nifi/controller/reporting/ClusteredReportingTaskNode.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/controller/reporting/ClusteredReportingTaskNode.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/main/java/org/apache/nifi/controller/reporting/ClusteredReportingTaskNode.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/controller/reporting/ClusteredReportingTaskNode.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/main/resources/nifi-cluster-manager-context.xml b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/resources/nifi-cluster-manager-context.xml similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/main/resources/nifi-cluster-manager-context.xml rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/resources/nifi-cluster-manager-context.xml diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/test/java/org/apache/nifi/cluster/event/impl/EventManagerImplTest.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/event/impl/EventManagerImplTest.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/test/java/org/apache/nifi/cluster/event/impl/EventManagerImplTest.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/event/impl/EventManagerImplTest.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/test/java/org/apache/nifi/cluster/firewall/impl/FileBasedClusterNodeFirewallTest.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/firewall/impl/FileBasedClusterNodeFirewallTest.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/test/java/org/apache/nifi/cluster/firewall/impl/FileBasedClusterNodeFirewallTest.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/firewall/impl/FileBasedClusterNodeFirewallTest.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/test/java/org/apache/nifi/cluster/flow/impl/DataFlowManagementServiceImplTest.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/flow/impl/DataFlowManagementServiceImplTest.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/test/java/org/apache/nifi/cluster/flow/impl/DataFlowManagementServiceImplTest.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/flow/impl/DataFlowManagementServiceImplTest.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/test/java/org/apache/nifi/cluster/manager/impl/HttpRequestReplicatorImplTest.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/manager/impl/HttpRequestReplicatorImplTest.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/test/java/org/apache/nifi/cluster/manager/impl/HttpRequestReplicatorImplTest.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/manager/impl/HttpRequestReplicatorImplTest.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/test/java/org/apache/nifi/cluster/manager/impl/HttpResponseMapperImplTest.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/manager/impl/HttpResponseMapperImplTest.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/test/java/org/apache/nifi/cluster/manager/impl/HttpResponseMapperImplTest.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/manager/impl/HttpResponseMapperImplTest.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/test/java/org/apache/nifi/cluster/manager/impl/TestWebClusterManager.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/manager/impl/TestWebClusterManager.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/test/java/org/apache/nifi/cluster/manager/impl/TestWebClusterManager.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/manager/impl/TestWebClusterManager.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/test/java/org/apache/nifi/cluster/manager/testutils/HttpRequest.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/manager/testutils/HttpRequest.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/test/java/org/apache/nifi/cluster/manager/testutils/HttpRequest.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/manager/testutils/HttpRequest.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/test/java/org/apache/nifi/cluster/manager/testutils/HttpResponse.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/manager/testutils/HttpResponse.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/test/java/org/apache/nifi/cluster/manager/testutils/HttpResponse.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/manager/testutils/HttpResponse.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/test/java/org/apache/nifi/cluster/manager/testutils/HttpResponseAction.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/manager/testutils/HttpResponseAction.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/test/java/org/apache/nifi/cluster/manager/testutils/HttpResponseAction.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/manager/testutils/HttpResponseAction.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/test/java/org/apache/nifi/cluster/manager/testutils/HttpServer.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/manager/testutils/HttpServer.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/test/java/org/apache/nifi/cluster/manager/testutils/HttpServer.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/manager/testutils/HttpServer.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/test/java/org/apache/nifi/cluster/protocol/impl/ClusterManagerProtocolSenderImplTest.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/protocol/impl/ClusterManagerProtocolSenderImplTest.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/test/java/org/apache/nifi/cluster/protocol/impl/ClusterManagerProtocolSenderImplTest.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/protocol/impl/ClusterManagerProtocolSenderImplTest.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/test/java/org/apache/nifi/cluster/protocol/impl/ClusterServiceLocatorTest.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/protocol/impl/ClusterServiceLocatorTest.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/test/java/org/apache/nifi/cluster/protocol/impl/ClusterServiceLocatorTest.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/protocol/impl/ClusterServiceLocatorTest.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/test/java/org/apache/nifi/cluster/protocol/impl/ClusterServicesBroadcasterTest.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/protocol/impl/ClusterServicesBroadcasterTest.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/test/java/org/apache/nifi/cluster/protocol/impl/ClusterServicesBroadcasterTest.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/protocol/impl/ClusterServicesBroadcasterTest.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/test/java/org/apache/nifi/cluster/protocol/impl/MulticastProtocolListenerTest.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/protocol/impl/MulticastProtocolListenerTest.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/test/java/org/apache/nifi/cluster/protocol/impl/MulticastProtocolListenerTest.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/protocol/impl/MulticastProtocolListenerTest.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/test/java/org/apache/nifi/cluster/protocol/impl/NodeProtocolSenderImplTest.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/protocol/impl/NodeProtocolSenderImplTest.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/test/java/org/apache/nifi/cluster/protocol/impl/NodeProtocolSenderImplTest.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/protocol/impl/NodeProtocolSenderImplTest.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/test/java/org/apache/nifi/cluster/protocol/impl/SocketProtocolListenerTest.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/protocol/impl/SocketProtocolListenerTest.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/test/java/org/apache/nifi/cluster/protocol/impl/SocketProtocolListenerTest.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/protocol/impl/SocketProtocolListenerTest.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/test/java/org/apache/nifi/cluster/protocol/testutils/DelayedProtocolHandler.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/protocol/testutils/DelayedProtocolHandler.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/test/java/org/apache/nifi/cluster/protocol/testutils/DelayedProtocolHandler.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/protocol/testutils/DelayedProtocolHandler.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/test/java/org/apache/nifi/cluster/protocol/testutils/ReflexiveProtocolHandler.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/protocol/testutils/ReflexiveProtocolHandler.java similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/test/java/org/apache/nifi/cluster/protocol/testutils/ReflexiveProtocolHandler.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/protocol/testutils/ReflexiveProtocolHandler.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/test/resources/logback-test.xml b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/resources/logback-test.xml similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/test/resources/logback-test.xml rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/resources/logback-test.xml diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/test/resources/org/apache/nifi/cluster/firewall/impl/empty.txt b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/resources/org/apache/nifi/cluster/firewall/impl/empty.txt similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/test/resources/org/apache/nifi/cluster/firewall/impl/empty.txt rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/resources/org/apache/nifi/cluster/firewall/impl/empty.txt diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/test/resources/org/apache/nifi/cluster/firewall/impl/ips.txt b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/resources/org/apache/nifi/cluster/firewall/impl/ips.txt similarity index 100% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/test/resources/org/apache/nifi/cluster/firewall/impl/ips.txt rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/resources/org/apache/nifi/cluster/firewall/impl/ips.txt diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/pom.xml b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/pom.xml index bb53affcfb..f2bf6c94c7 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/pom.xml +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/pom.xml @@ -64,8 +64,8 @@ nifi-client-dto - org.apache.nifi - nifi-site-to-site-client + org.apache.nifi + nifi-site-to-site-client org.quartz-scheduler @@ -121,4 +121,19 @@ test + + + + org.apache.rat + apache-rat-plugin + + + src/test/resources/conf/0bytes.xml + src/test/resources/conf/termination-only.xml + src/test/resources/hello.txt + + + + + diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/pom.xml b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/pom.xml index 60eda8a480..6b587fd829 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/pom.xml +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/pom.xml @@ -588,6 +588,19 @@ + + org.apache.rat + apache-rat-plugin + + + src/main/webapp/js/json2.js + src/main/webapp/js/jquery/ + src/main/webapp/js/d3/d3.min.js + src/main/webapp/js/codemirror/ + src/main/webapp/css/reset.css + + + diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/pom.xml b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/pom.xml index 544dd7784c..2aaa045407 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/pom.xml +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/pom.xml @@ -30,9 +30,9 @@ nifi-security nifi-site-to-site nifi-framework-core - nifi-cluster-protocol - nifi-cluster-web - nifi-cluster + nifi-framework-cluster-protocol + nifi-framework-cluster-web + nifi-framework-cluster nifi-file-authorization-provider nifi-cluster-authorization-provider nifi-user-actions diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/pom.xml b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/pom.xml index 2043a0d081..170a95e71c 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/pom.xml +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/pom.xml @@ -48,7 +48,7 @@ org.apache.nifi nifi-distributed-cache-client-service-api - + org.apache.nifi nifi-http-context-map-api @@ -169,4 +169,88 @@ 1.7 + + + + org.apache.rat + apache-rat-plugin + + + src/test/resources/hello.txt + src/test/resources/CharacterSetConversionSamples/Converted.txt + src/test/resources/CharacterSetConversionSamples/Original.txt + src/test/resources/CompressedData/SampleFile.txt + src/test/resources/CompressedData/SampleFileConcat.txt + src/test/resources/ExecuteCommand/1000bytes.txt + src/test/resources/ExecuteCommand/test.txt + src/test/resources/ScanAttribute/dictionary-with-empty-new-lines + src/test/resources/ScanAttribute/dictionary-with-extra-info + src/test/resources/ScanAttribute/dictionary1 + src/test/resources/TestIdentifyMimeType/1.txt + src/test/resources/TestJson/json-sample.json + src/test/resources/TestMergeContent/demarcate + src/test/resources/TestMergeContent/foot + src/test/resources/TestMergeContent/head + src/test/resources/TestModifyBytes/noFooter.txt + src/test/resources/TestModifyBytes/noFooter_noHeader.txt + src/test/resources/TestModifyBytes/noHeader.txt + src/test/resources/TestModifyBytes/testFile.txt + src/test/resources/TestReplaceTextLineByLine/$1$1.txt + src/test/resources/TestReplaceTextLineByLine/BRue_cRue_RiRey.txt + src/test/resources/TestReplaceTextLineByLine/Blu$2e_clu$2e.txt + src/test/resources/TestReplaceTextLineByLine/D$d_h$d.txt + src/test/resources/TestReplaceTextLineByLine/Good.txt + src/test/resources/TestReplaceTextLineByLine/Spider.txt + src/test/resources/TestReplaceTextLineByLine/[DODO].txt + src/test/resources/TestReplaceTextLineByLine/cu[$1]_Po[$1].txt + src/test/resources/TestReplaceTextLineByLine/cu_Po.txt + src/test/resources/TestReplaceTextLineByLine/food.txt + src/test/resources/TestReplaceTextLineByLine/testFile.txt + src/test/resources/TestReplaceTextWithMapping/color-fruit-backreference-mapping.txt + src/test/resources/TestReplaceTextWithMapping/color-fruit-blank-mapping.txt + src/test/resources/TestReplaceTextWithMapping/color-fruit-escaped-dollar-mapping.txt + src/test/resources/TestReplaceTextWithMapping/color-fruit-excessive-backreference-mapping-simple.txt + src/test/resources/TestReplaceTextWithMapping/color-fruit-excessive-backreference-mapping.txt + src/test/resources/TestReplaceTextWithMapping/color-fruit-invalid-backreference-mapping.txt + src/test/resources/TestReplaceTextWithMapping/color-fruit-mapping.txt + src/test/resources/TestReplaceTextWithMapping/color-fruit-no-match-mapping.txt + src/test/resources/TestReplaceTextWithMapping/color-fruit-space-mapping.txt + src/test/resources/TestReplaceTextWithMapping/colors-without-dashes.txt + src/test/resources/TestReplaceTextWithMapping/colors.txt + src/test/resources/TestScanContent/helloWorld + src/test/resources/TestScanContent/wellthengood-bye + src/test/resources/TestSplitText/1.txt + src/test/resources/TestSplitText/2.txt + src/test/resources/TestSplitText/3.txt + src/test/resources/TestSplitText/4.txt + src/test/resources/TestSplitText/5.txt + src/test/resources/TestSplitText/6.txt + src/test/resources/TestSplitText/original.txt + src/test/resources/TestTransformXml/math.html + src/test/resources/TestTransformXml/tokens.csv + src/test/resources/TestTransformXml/tokens.xml + src/test/resources/TestUnpackContent/folder/cal.txt + src/test/resources/TestUnpackContent/folder/date.txt + src/test/resources/TestXml/xml-bundle-1 + src/test/resources/CompressedData/SampleFile.txt.bz2 + src/test/resources/CompressedData/SampleFile.txt.gz + src/test/resources/CompressedData/SampleFile1.txt.bz2 + src/test/resources/CompressedData/SampleFile1.txt.gz + src/test/resources/CompressedData/SampleFileConcat.txt.bz2 + src/test/resources/ExecuteCommand/TestIngestAndUpdate.jar + src/test/resources/ExecuteCommand/TestSuccess.jar + src/test/resources/TestIdentifyMimeType/1.jar + src/test/resources/TestIdentifyMimeType/1.tar + src/test/resources/TestIdentifyMimeType/1.tar.gz + src/test/resources/TestIdentifyMimeType/1.txt.bz2 + src/test/resources/TestIdentifyMimeType/1.txt.gz + src/test/resources/TestIdentifyMimeType/1.zip + src/test/resources/TestIdentifyMimeType/flowfilev1.tar + src/test/resources/TestUnpackContent/data.tar + src/test/resources/TestUnpackContent/data.zip + + + + + diff --git a/nifi/pom.xml b/nifi/pom.xml index 72838bd1c3..8ba4108c9d 100644 --- a/nifi/pom.xml +++ b/nifi/pom.xml @@ -65,7 +65,7 @@ nifi-assembly nifi-docs nifi-maven-archetypes - nifi-external + nifi-external scm:git:git://git.apache.org/incubator-nifi.git @@ -538,10 +538,10 @@ ${hadoop.version} - org.apache.avro - avro - 1.7.6 - + org.apache.avro + avro + 1.7.6 + com.sun.jersey @@ -654,11 +654,11 @@ nifi-utils 0.0.3-incubating-SNAPSHOT - - org.apache.nifi - nifi-site-to-site-client - 0.0.3-incubating-SNAPSHOT - + + org.apache.nifi + nifi-site-to-site-client + 0.0.3-incubating-SNAPSHOT + org.apache.nifi nifi-web-utils @@ -775,7 +775,7 @@ 0.0.3-incubating-SNAPSHOT nar - + org.apache.nifi nifi-kite-nar 0.0.3-incubating-SNAPSHOT @@ -787,7 +787,7 @@ 0.0.3-incubating-SNAPSHOT nar - + org.apache.nifi nifi-http-context-map-nar 0.0.3-incubating-SNAPSHOT @@ -1008,15 +1008,9 @@ apache-rat-plugin - src/main/webapp/js/json2.js - src/main/webapp/js/jquery/ - src/main/webapp/js/d3/d3.min.js - src/main/webapp/js/codemirror/ - src/main/webapp/css/reset.css - src/test/resources/ - nb-configuration.xml - nbactions.xml - DEPENDENCIES + nb-configuration.xml + nbactions.xml + DEPENDENCIES From 37645b599ec49367e874e726b6b7dadf5234a4bd Mon Sep 17 00:00:00 2001 From: Ryan Blue Date: Mon, 16 Mar 2015 20:53:39 -0700 Subject: [PATCH 21/26] NIFI-428: Update to Kite 1.0.0. Kite 1.0.0 fixes the JSON and CSV readers so they no longer need to be in NiFi. --- .../nifi-kite-processors/pom.xml | 2 +- .../kite/AbstractKiteProcessor.java | 8 +- .../processors/kite/ConvertCSVToAvro.java | 25 ++- .../processors/kite/ConvertJSONToAvro.java | 17 +- .../nifi/processors/kite/JSONFileReader.java | 114 ------------ .../processors/kite/StoreInKiteDataset.java | 7 +- .../spi/filesystem/CSVFileReaderFixed.java | 172 ------------------ .../nifi/processors/kite/TestGetSchema.java | 3 + 8 files changed, 39 insertions(+), 309 deletions(-) delete mode 100644 nifi/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-processors/src/main/java/org/apache/nifi/processors/kite/JSONFileReader.java delete mode 100644 nifi/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-processors/src/main/java/org/kitesdk/data/spi/filesystem/CSVFileReaderFixed.java diff --git a/nifi/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-processors/pom.xml b/nifi/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-processors/pom.xml index a01998a6c1..6cb1c7a3c0 100644 --- a/nifi/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-processors/pom.xml +++ b/nifi/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-processors/pom.xml @@ -25,7 +25,7 @@ jar - 0.18.0 + 1.0.0 11.0.2 4.10 1.3.9-1 diff --git a/nifi/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-processors/src/main/java/org/apache/nifi/processors/kite/AbstractKiteProcessor.java b/nifi/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-processors/src/main/java/org/apache/nifi/processors/kite/AbstractKiteProcessor.java index 98329ff4ea..2113854ab0 100644 --- a/nifi/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-processors/src/main/java/org/apache/nifi/processors/kite/AbstractKiteProcessor.java +++ b/nifi/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-processors/src/main/java/org/apache/nifi/processors/kite/AbstractKiteProcessor.java @@ -84,7 +84,7 @@ abstract class AbstractKiteProcessor extends AbstractProcessor { boolean isValid = true; if (uri == null || uri.isEmpty()) { isValid = false; - } else { + } else if (!uri.contains("$")) { try { new URIBuilder(URI.create(uri)).build(); } catch (RuntimeException e) { @@ -161,10 +161,12 @@ abstract class AbstractKiteProcessor extends AbstractProcessor { context.getProperty(CONF_XML_FILES).getValue()); String error = null; - try { + if (!uri.contains("$")) { + try { getSchema(uri, conf); - } catch (SchemaNotFoundException e) { + } catch (SchemaNotFoundException e) { error = e.getMessage(); + } } return new ValidationResult.Builder() .subject(subject) diff --git a/nifi/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-processors/src/main/java/org/apache/nifi/processors/kite/ConvertCSVToAvro.java b/nifi/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-processors/src/main/java/org/apache/nifi/processors/kite/ConvertCSVToAvro.java index 5181bbe187..be8c416fcd 100644 --- a/nifi/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-processors/src/main/java/org/apache/nifi/processors/kite/ConvertCSVToAvro.java +++ b/nifi/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-processors/src/main/java/org/apache/nifi/processors/kite/ConvertCSVToAvro.java @@ -48,8 +48,9 @@ import org.apache.nifi.processor.util.StandardValidators; import org.kitesdk.data.DatasetException; import org.kitesdk.data.DatasetIOException; import org.kitesdk.data.DatasetRecordException; +import org.kitesdk.data.SchemaNotFoundException; import org.kitesdk.data.spi.DefaultConfiguration; -import org.kitesdk.data.spi.filesystem.CSVFileReaderFixed; +import org.kitesdk.data.spi.filesystem.CSVFileReader; import org.kitesdk.data.spi.filesystem.CSVProperties; import static org.apache.nifi.processor.util.StandardValidators.createLongValidator; @@ -90,6 +91,7 @@ public class ConvertCSVToAvro extends AbstractKiteProcessor { .description( "Outgoing Avro schema for each record created from a CSV row") .addValidator(SCHEMA_VALIDATOR) + .expressionLanguageSupported(true) .required(true) .build(); @@ -201,9 +203,17 @@ public class ConvertCSVToAvro extends AbstractKiteProcessor { return; } - final Schema schema = getSchema( - context.getProperty(SCHEMA).getValue(), - DefaultConfiguration.get()); + String schemaProperty = context.getProperty(SCHEMA) + .evaluateAttributeExpressions(flowFile) + .getValue(); + final Schema schema; + try { + schema = getSchema(schemaProperty, DefaultConfiguration.get()); + } catch (SchemaNotFoundException e) { + getLogger().error("Cannot find schema: " + schemaProperty); + session.transfer(flowFile, FAILURE); + return; + } final DataFileWriter writer = new DataFileWriter<>( AvroUtil.newDatumWriter(schema, Record.class)); @@ -215,7 +225,7 @@ public class ConvertCSVToAvro extends AbstractKiteProcessor { public void process(InputStream in, OutputStream out) throws IOException { long written = 0L; long errors = 0L; - try (CSVFileReaderFixed reader = new CSVFileReaderFixed<>( + try (CSVFileReader reader = new CSVFileReader<>( in, props, schema, Record.class)) { reader.initialize(); try (DataFileWriter w = writer.create(schema, out)) { @@ -247,11 +257,6 @@ public class ConvertCSVToAvro extends AbstractKiteProcessor { } catch (DatasetException e) { getLogger().error("Failed to read FlowFile", e); session.transfer(flowFile, FAILURE); - - } catch (Throwable t) { - getLogger().error("Unknown Throwable", t); - session.rollback(true); // penalize just in case - context.yield(); } } } diff --git a/nifi/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-processors/src/main/java/org/apache/nifi/processors/kite/ConvertJSONToAvro.java b/nifi/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-processors/src/main/java/org/apache/nifi/processors/kite/ConvertJSONToAvro.java index 8eeea1a9c4..7e40f14362 100644 --- a/nifi/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-processors/src/main/java/org/apache/nifi/processors/kite/ConvertJSONToAvro.java +++ b/nifi/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-processors/src/main/java/org/apache/nifi/processors/kite/ConvertJSONToAvro.java @@ -42,7 +42,9 @@ import org.apache.nifi.processor.io.StreamCallback; import org.kitesdk.data.DatasetException; import org.kitesdk.data.DatasetIOException; import org.kitesdk.data.DatasetRecordException; +import org.kitesdk.data.SchemaNotFoundException; import org.kitesdk.data.spi.DefaultConfiguration; +import org.kitesdk.data.spi.filesystem.JSONFileReader; @Tags({"kite", "json", "avro"}) @CapabilityDescription( @@ -66,6 +68,7 @@ public class ConvertJSONToAvro extends AbstractKiteProcessor { .description( "Outgoing Avro schema for each record created from a JSON object") .addValidator(SCHEMA_VALIDATOR) + .expressionLanguageSupported(true) .required(true) .build(); @@ -102,9 +105,17 @@ public class ConvertJSONToAvro extends AbstractKiteProcessor { return; } - final Schema schema = getSchema( - context.getProperty(SCHEMA).getValue(), - DefaultConfiguration.get()); + String schemaProperty = context.getProperty(SCHEMA) + .evaluateAttributeExpressions(flowFile) + .getValue(); + final Schema schema; + try { + schema = getSchema(schemaProperty, DefaultConfiguration.get()); + } catch (SchemaNotFoundException e) { + getLogger().error("Cannot find schema: " + schemaProperty); + session.transfer(flowFile, FAILURE); + return; + } final DataFileWriter writer = new DataFileWriter<>( AvroUtil.newDatumWriter(schema, Record.class)); diff --git a/nifi/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-processors/src/main/java/org/apache/nifi/processors/kite/JSONFileReader.java b/nifi/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-processors/src/main/java/org/apache/nifi/processors/kite/JSONFileReader.java deleted file mode 100644 index 7bb7f3df5d..0000000000 --- a/nifi/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-processors/src/main/java/org/apache/nifi/processors/kite/JSONFileReader.java +++ /dev/null @@ -1,114 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.nifi.processors.kite; - -import com.fasterxml.jackson.databind.JsonNode; -import com.google.common.base.Function; -import com.google.common.base.Preconditions; -import com.google.common.collect.Iterators; -import java.io.IOException; -import java.io.InputStream; -import java.util.Iterator; -import javax.annotation.Nullable; -import org.apache.avro.Schema; -import org.apache.avro.generic.GenericData; -import org.kitesdk.data.DatasetIOException; -import org.kitesdk.data.spi.AbstractDatasetReader; -import org.kitesdk.data.spi.DataModelUtil; -import org.kitesdk.data.spi.JsonUtil; -import org.kitesdk.data.spi.ReaderWriterState; - -/** - * This is a temporary addition. The version in 0.18.0 throws a NPE when the - * InputStream constructor is used. - */ -class JSONFileReader extends AbstractDatasetReader { - - private final GenericData model; - private final Schema schema; - - private InputStream incoming = null; - - // state - private ReaderWriterState state = ReaderWriterState.NEW; - private Iterator iterator; - - public JSONFileReader(InputStream incoming, Schema schema, Class type) { - this.incoming = incoming; - this.schema = schema; - this.model = DataModelUtil.getDataModelForType(type); - this.state = ReaderWriterState.NEW; - } - - @Override - public void initialize() { - Preconditions.checkState(state.equals(ReaderWriterState.NEW), - "A reader may not be opened more than once - current state:%s", state); - Preconditions.checkArgument(Schema.Type.RECORD.equals(schema.getType()), - "Schemas for JSON files should be record"); - - this.iterator = Iterators.transform(JsonUtil.parser(incoming), - new Function() { - @Override - @SuppressWarnings("unchecked") - public E apply(@Nullable JsonNode node) { - return (E) JsonUtil.convertToAvro(model, node, schema); - } - }); - - this.state = ReaderWriterState.OPEN; - } - - @Override - public boolean hasNext() { - Preconditions.checkState(state.equals(ReaderWriterState.OPEN), - "Attempt to read from a file in state:%s", state); - return iterator.hasNext(); - } - - @Override - public E next() { - Preconditions.checkState(state.equals(ReaderWriterState.OPEN), - "Attempt to read from a file in state:%s", state); - return iterator.next(); - } - - @Override - public void close() { - if (!state.equals(ReaderWriterState.OPEN)) { - return; - } - - iterator = null; - try { - incoming.close(); - } catch (IOException e) { - throw new DatasetIOException("Unable to close reader path", e); - } - - state = ReaderWriterState.CLOSED; - } - - @Override - public boolean isOpen() { - return (this.state == ReaderWriterState.OPEN); - } - -} diff --git a/nifi/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-processors/src/main/java/org/apache/nifi/processors/kite/StoreInKiteDataset.java b/nifi/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-processors/src/main/java/org/apache/nifi/processors/kite/StoreInKiteDataset.java index 314f0083e6..f65e0bce7c 100644 --- a/nifi/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-processors/src/main/java/org/apache/nifi/processors/kite/StoreInKiteDataset.java +++ b/nifi/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-processors/src/main/java/org/apache/nifi/processors/kite/StoreInKiteDataset.java @@ -47,7 +47,7 @@ import org.kitesdk.data.ValidationException; import org.kitesdk.data.View; import org.kitesdk.data.spi.SchemaValidationUtil; -@Tags({"kite", "avro", "parquet", "hive", "hdfs", "hbase"}) +@Tags({"kite", "avro", "parquet", "hadoop", "hive", "hdfs", "hbase"}) @CapabilityDescription("Stores Avro records in a Kite dataset") public class StoreInKiteDataset extends AbstractKiteProcessor { private static final Relationship SUCCESS = new Relationship.Builder() @@ -151,11 +151,6 @@ public class StoreInKiteDataset extends AbstractKiteProcessor { getLogger().error(e.getMessage()); getLogger().debug("Incompatible schema error", e); session.transfer(flowFile, INCOMPATIBLE); - - } catch (Throwable t) { - getLogger().error("Unknown Throwable", t); - session.rollback(true); // penalize just in case - context.yield(); } } diff --git a/nifi/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-processors/src/main/java/org/kitesdk/data/spi/filesystem/CSVFileReaderFixed.java b/nifi/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-processors/src/main/java/org/kitesdk/data/spi/filesystem/CSVFileReaderFixed.java deleted file mode 100644 index d6c200646b..0000000000 --- a/nifi/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-processors/src/main/java/org/kitesdk/data/spi/filesystem/CSVFileReaderFixed.java +++ /dev/null @@ -1,172 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.kitesdk.data.spi.filesystem; - -import au.com.bytecode.opencsv.CSVReader; -import java.io.InputStream; -import com.google.common.collect.Lists; -import java.util.List; -import org.apache.hadoop.mapreduce.InputSplit; -import org.apache.hadoop.mapreduce.RecordReader; -import org.apache.hadoop.mapreduce.TaskAttemptContext; -import org.kitesdk.data.DatasetDescriptor; -import org.kitesdk.data.DatasetIOException; -import org.kitesdk.data.spi.AbstractDatasetReader; -import org.kitesdk.data.spi.DescriptorUtil; -import org.kitesdk.data.spi.EntityAccessor; -import org.kitesdk.data.spi.ReaderWriterState; -import com.google.common.base.Preconditions; -import org.apache.avro.Schema; -import org.apache.hadoop.fs.FSDataInputStream; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.kitesdk.data.spi.filesystem.CSVProperties; -import org.kitesdk.data.spi.filesystem.CSVUtil; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.IOException; -import java.util.NoSuchElementException; - -import static org.kitesdk.data.spi.filesystem.FileSystemProperties.REUSE_RECORDS; - -/** - * This is a temporary addition. The version in 0.18.0 throws a NPE when the - * InputStream constructor is used. - */ -public class CSVFileReaderFixed extends AbstractDatasetReader { - - private final CSVProperties props; - private final Schema schema; - private final boolean reuseRecords; - - private final Class recordClass; - - private CSVReader reader = null; - private CSVRecordBuilder builder; - - private InputStream incoming = null; - - // state - private ReaderWriterState state = ReaderWriterState.NEW; - private boolean hasNext = false; - private String[] next = null; - private E record = null; - - public CSVFileReaderFixed(InputStream incoming, CSVProperties props, - Schema schema, Class type) { - this.incoming = incoming; - this.schema = schema; - this.recordClass = type; - this.state = ReaderWriterState.NEW; - this.props = props; - this.reuseRecords = false; - - Preconditions.checkArgument(Schema.Type.RECORD.equals(schema.getType()), - "Schemas for CSV files must be records of primitive types"); - } - - @Override - @SuppressWarnings("unchecked") - public void initialize() { - Preconditions.checkState(state.equals(ReaderWriterState.NEW), - "A reader may not be opened more than once - current state:%s", state); - - this.reader = CSVUtil.newReader(incoming, props); - - List header = null; - if (props.useHeader) { - this.hasNext = advance(); - header = Lists.newArrayList(next); - } else if (props.header != null) { - try { - header = Lists.newArrayList( - CSVUtil.newParser(props).parseLine(props.header)); - } catch (IOException e) { - throw new DatasetIOException( - "Failed to parse header from properties: " + props.header, e); - } - } - - this.builder = new CSVRecordBuilder(schema, recordClass, header); - - // initialize by reading the first record - this.hasNext = advance(); - - this.state = ReaderWriterState.OPEN; - } - - @Override - public boolean hasNext() { - Preconditions.checkState(state.equals(ReaderWriterState.OPEN), - "Attempt to read from a file in state:%s", state); - return hasNext; - } - - @Override - public E next() { - Preconditions.checkState(state.equals(ReaderWriterState.OPEN), - "Attempt to read from a file in state:%s", state); - - if (!hasNext) { - throw new NoSuchElementException(); - } - - try { - if (reuseRecords) { - this.record = builder.makeRecord(next, record); - return record; - } else { - return builder.makeRecord(next, null); - } - } finally { - this.hasNext = advance(); - } - } - - private boolean advance() { - try { - next = reader.readNext(); - } catch (IOException e) { - throw new DatasetIOException("Could not read record", e); - } - return (next != null); - } - - @Override - public void close() { - if (!state.equals(ReaderWriterState.OPEN)) { - return; - } - - try { - reader.close(); - } catch (IOException e) { - throw new DatasetIOException("Unable to close reader", e); - } - - state = ReaderWriterState.CLOSED; - } - - @Override - public boolean isOpen() { - return (this.state == ReaderWriterState.OPEN); - } -} diff --git a/nifi/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-processors/src/test/java/org/apache/nifi/processors/kite/TestGetSchema.java b/nifi/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-processors/src/test/java/org/apache/nifi/processors/kite/TestGetSchema.java index 2dd51d32a7..a3489ec1ab 100644 --- a/nifi/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-processors/src/test/java/org/apache/nifi/processors/kite/TestGetSchema.java +++ b/nifi/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-processors/src/test/java/org/apache/nifi/processors/kite/TestGetSchema.java @@ -66,6 +66,9 @@ public class TestGetSchema { @Ignore("Does not work on windows") public void testSchemaFromKiteURIs() throws IOException { String location = temp.newFolder("ns", "temp").toString(); + if (location.endsWith("/")) { + location = location.substring(0, location.length() - 1); + } String datasetUri = "dataset:" + location; DatasetDescriptor descriptor = new DatasetDescriptor.Builder() .schema(SCHEMA) From e40cd97bb45ec7362dacf0a82ac94cf812e814b4 Mon Sep 17 00:00:00 2001 From: joewitt Date: Tue, 17 Mar 2015 10:33:44 -0400 Subject: [PATCH 22/26] NIFI-429 bumped all versions to next planned release which is 0.1.0 --- nifi/nifi-api/pom.xml | 2 +- nifi/nifi-assembly/pom.xml | 2 +- nifi/nifi-bootstrap/pom.xml | 2 +- .../nifi-data-provenance-utils/pom.xml | 2 +- .../nifi-expression-language/pom.xml | 2 +- .../nifi-flowfile-packager/pom.xml | 2 +- nifi/nifi-commons/nifi-logging-utils/pom.xml | 2 +- .../nifi-processor-utilities/pom.xml | 2 +- nifi/nifi-commons/nifi-properties/pom.xml | 2 +- nifi/nifi-commons/nifi-security-utils/pom.xml | 2 +- .../nifi-site-to-site-client/pom.xml | 4 +- nifi/nifi-commons/nifi-socket-utils/pom.xml | 2 +- nifi/nifi-commons/nifi-utils/pom.xml | 4 +- nifi/nifi-commons/nifi-web-utils/pom.xml | 2 +- .../nifi-commons/nifi-write-ahead-log/pom.xml | 2 +- nifi/nifi-commons/pom.xml | 2 +- nifi/nifi-docs/pom.xml | 2 +- .../nifi-external/nifi-spark-receiver/pom.xml | 2 +- nifi/nifi-external/pom.xml | 2 +- .../nifi-processor-bundle-archetype/pom.xml | 2 +- nifi/nifi-maven-archetypes/pom.xml | 2 +- nifi/nifi-mock/pom.xml | 2 +- .../nifi-framework-nar/pom.xml | 2 +- .../nifi-administration/pom.xml | 2 +- .../nifi-framework/nifi-client-dto/pom.xml | 2 +- .../pom.xml | 2 +- .../nifi-file-authorization-provider/pom.xml | 2 +- .../nifi-framework-cluster-protocol/pom.xml | 2 +- .../nifi-framework-cluster-web/pom.xml | 2 +- .../nifi-framework-cluster/pom.xml | 2 +- .../nifi-framework-core-api/pom.xml | 2 +- .../nifi-framework-core/pom.xml | 2 +- .../nifi-framework/nifi-nar-utils/pom.xml | 2 +- .../nifi-framework/nifi-resources/pom.xml | 2 +- .../nifi-framework/nifi-runtime/pom.xml | 2 +- .../nifi-framework/nifi-security/pom.xml | 2 +- .../nifi-framework/nifi-site-to-site/pom.xml | 2 +- .../nifi-framework/nifi-user-actions/pom.xml | 2 +- .../nifi-web/nifi-custom-ui-utilities/pom.xml | 2 +- .../nifi-web/nifi-jetty/pom.xml | 2 +- .../nifi-web/nifi-web-api/pom.xml | 4 +- .../nifi-web/nifi-web-docs/pom.xml | 2 +- .../nifi-web/nifi-web-error/pom.xml | 2 +- .../nifi-web-optimistic-locking/pom.xml | 2 +- .../nifi-web/nifi-web-security/pom.xml | 2 +- .../nifi-web/nifi-web-ui/pom.xml | 2 +- .../nifi-framework/nifi-web/pom.xml | 10 +-- .../nifi-framework/pom.xml | 2 +- .../nifi-framework-bundle/pom.xml | 34 +++++----- .../nifi-hadoop-nar/pom.xml | 2 +- .../nifi-hdfs-processors/pom.xml | 2 +- .../nifi-hadoop-bundle/pom.xml | 4 +- .../nifi-hadoop-libraries-nar/pom.xml | 2 +- .../nifi-hadoop-libraries-bundle/pom.xml | 2 +- .../nifi-jetty-bundle/pom.xml | 2 +- .../nifi-kafka-bundle/nifi-kafka-nar/pom.xml | 2 +- .../nifi-kafka-processors/pom.xml | 2 +- .../nifi-kafka-bundle/pom.xml | 4 +- .../nifi-kite-bundle/nifi-kite-nar/pom.xml | 2 +- .../nifi-kite-processors/pom.xml | 2 +- .../nifi-nar-bundles/nifi-kite-bundle/pom.xml | 4 +- .../pom.xml | 2 +- .../nifi-provenance-repository-nar/pom.xml | 2 +- .../pom.xml | 2 +- .../nifi-provenance-repository-bundle/pom.xml | 6 +- .../nifi-standard-nar/pom.xml | 2 +- .../nifi-standard-prioritizers/pom.xml | 2 +- .../nifi-standard-processors/pom.xml | 2 +- .../nifi-standard-reporting-tasks/pom.xml | 2 +- .../nifi-standard-bundle/pom.xml | 8 +-- .../pom.xml | 2 +- .../pom.xml | 2 +- .../nifi-distributed-cache-protocol/pom.xml | 2 +- .../nifi-distributed-cache-server/pom.xml | 2 +- .../pom.xml | 2 +- .../pom.xml | 2 +- .../nifi-http-context-map-api/pom.xml | 2 +- .../nifi-http-context-map-nar/pom.xml | 2 +- .../nifi-http-context-map/pom.xml | 2 +- .../nifi-http-context-map-bundle/pom.xml | 2 +- .../pom.xml | 2 +- .../nifi-ssl-context-nar/pom.xml | 2 +- .../nifi-ssl-context-service/pom.xml | 2 +- .../nifi-ssl-context-bundle/pom.xml | 2 +- .../nifi-ssl-context-service-api/pom.xml | 2 +- .../nifi-standard-services-api-nar/pom.xml | 2 +- .../nifi-standard-services/pom.xml | 2 +- .../nifi-update-attribute-model/pom.xml | 2 +- .../nifi-update-attribute-nar/pom.xml | 2 +- .../nifi-update-attribute-processor/pom.xml | 2 +- .../nifi-update-attribute-ui/pom.xml | 2 +- .../nifi-update-attribute-bundle/pom.xml | 8 +-- nifi/nifi-nar-bundles/pom.xml | 30 ++++---- nifi/pom.xml | 68 +++++++++---------- 94 files changed, 175 insertions(+), 175 deletions(-) diff --git a/nifi/nifi-api/pom.xml b/nifi/nifi-api/pom.xml index 6105671e9b..0321a43c80 100644 --- a/nifi/nifi-api/pom.xml +++ b/nifi/nifi-api/pom.xml @@ -18,7 +18,7 @@ org.apache.nifi nifi - 0.0.3-incubating-SNAPSHOT + 0.1.0-incubating-SNAPSHOT nifi-api jar diff --git a/nifi/nifi-assembly/pom.xml b/nifi/nifi-assembly/pom.xml index 200a9e0b8f..ece7dbb21a 100644 --- a/nifi/nifi-assembly/pom.xml +++ b/nifi/nifi-assembly/pom.xml @@ -18,7 +18,7 @@ org.apache.nifi nifi - 0.0.3-incubating-SNAPSHOT + 0.1.0-incubating-SNAPSHOT nifi-assembly pom diff --git a/nifi/nifi-bootstrap/pom.xml b/nifi/nifi-bootstrap/pom.xml index b2188a8efa..489f491537 100644 --- a/nifi/nifi-bootstrap/pom.xml +++ b/nifi/nifi-bootstrap/pom.xml @@ -17,7 +17,7 @@ org.apache.nifi nifi - 0.0.3-incubating-SNAPSHOT + 0.1.0-incubating-SNAPSHOT nifi-bootstrap jar diff --git a/nifi/nifi-commons/nifi-data-provenance-utils/pom.xml b/nifi/nifi-commons/nifi-data-provenance-utils/pom.xml index a7cdf930d9..de4fc93157 100644 --- a/nifi/nifi-commons/nifi-data-provenance-utils/pom.xml +++ b/nifi/nifi-commons/nifi-data-provenance-utils/pom.xml @@ -17,7 +17,7 @@ org.apache.nifi nifi-commons - 0.0.3-incubating-SNAPSHOT + 0.1.0-incubating-SNAPSHOT nifi-data-provenance-utils jar diff --git a/nifi/nifi-commons/nifi-expression-language/pom.xml b/nifi/nifi-commons/nifi-expression-language/pom.xml index e94f1f6eb6..3521b55ebf 100644 --- a/nifi/nifi-commons/nifi-expression-language/pom.xml +++ b/nifi/nifi-commons/nifi-expression-language/pom.xml @@ -17,7 +17,7 @@ org.apache.nifi nifi-commons - 0.0.3-incubating-SNAPSHOT + 0.1.0-incubating-SNAPSHOT nifi-expression-language jar diff --git a/nifi/nifi-commons/nifi-flowfile-packager/pom.xml b/nifi/nifi-commons/nifi-flowfile-packager/pom.xml index 5543d1c3eb..a1288e9213 100644 --- a/nifi/nifi-commons/nifi-flowfile-packager/pom.xml +++ b/nifi/nifi-commons/nifi-flowfile-packager/pom.xml @@ -17,7 +17,7 @@ org.apache.nifi nifi-commons - 0.0.3-incubating-SNAPSHOT + 0.1.0-incubating-SNAPSHOT nifi-flowfile-packager jar diff --git a/nifi/nifi-commons/nifi-logging-utils/pom.xml b/nifi/nifi-commons/nifi-logging-utils/pom.xml index 99ab542b80..ea7cff9cee 100644 --- a/nifi/nifi-commons/nifi-logging-utils/pom.xml +++ b/nifi/nifi-commons/nifi-logging-utils/pom.xml @@ -18,7 +18,7 @@ org.apache.nifi nifi-commons - 0.0.3-incubating-SNAPSHOT + 0.1.0-incubating-SNAPSHOT nifi-logging-utils Utilities for logging diff --git a/nifi/nifi-commons/nifi-processor-utilities/pom.xml b/nifi/nifi-commons/nifi-processor-utilities/pom.xml index e3b684fe05..8034e828f4 100644 --- a/nifi/nifi-commons/nifi-processor-utilities/pom.xml +++ b/nifi/nifi-commons/nifi-processor-utilities/pom.xml @@ -18,7 +18,7 @@ org.apache.nifi nifi-commons - 0.0.3-incubating-SNAPSHOT + 0.1.0-incubating-SNAPSHOT nifi-processor-utils jar diff --git a/nifi/nifi-commons/nifi-properties/pom.xml b/nifi/nifi-commons/nifi-properties/pom.xml index 6bfd82eef9..5f6853a674 100644 --- a/nifi/nifi-commons/nifi-properties/pom.xml +++ b/nifi/nifi-commons/nifi-properties/pom.xml @@ -18,7 +18,7 @@ org.apache.nifi nifi-commons - 0.0.3-incubating-SNAPSHOT + 0.1.0-incubating-SNAPSHOT nifi-properties diff --git a/nifi/nifi-commons/nifi-security-utils/pom.xml b/nifi/nifi-commons/nifi-security-utils/pom.xml index 7bf1b4aecc..91912a56d5 100644 --- a/nifi/nifi-commons/nifi-security-utils/pom.xml +++ b/nifi/nifi-commons/nifi-security-utils/pom.xml @@ -17,7 +17,7 @@ org.apache.nifi nifi-commons - 0.0.3-incubating-SNAPSHOT + 0.1.0-incubating-SNAPSHOT nifi-security-utils Contains security functionality. diff --git a/nifi/nifi-commons/nifi-site-to-site-client/pom.xml b/nifi/nifi-commons/nifi-site-to-site-client/pom.xml index 76bfd4cb11..5d3d93ea31 100644 --- a/nifi/nifi-commons/nifi-site-to-site-client/pom.xml +++ b/nifi/nifi-commons/nifi-site-to-site-client/pom.xml @@ -19,7 +19,7 @@ org.apache.nifi nifi-commons - 0.0.3-incubating-SNAPSHOT + 0.1.0-incubating-SNAPSHOT nifi-site-to-site-client @@ -42,7 +42,7 @@ org.apache.nifi nifi-client-dto - 0.0.3-incubating-SNAPSHOT + 0.1.0-incubating-SNAPSHOT diff --git a/nifi/nifi-commons/nifi-socket-utils/pom.xml b/nifi/nifi-commons/nifi-socket-utils/pom.xml index f2f3397c94..49bede8488 100644 --- a/nifi/nifi-commons/nifi-socket-utils/pom.xml +++ b/nifi/nifi-commons/nifi-socket-utils/pom.xml @@ -18,7 +18,7 @@ org.apache.nifi nifi-commons - 0.0.3-incubating-SNAPSHOT + 0.1.0-incubating-SNAPSHOT nifi-socket-utils Utilities for socket communication diff --git a/nifi/nifi-commons/nifi-utils/pom.xml b/nifi/nifi-commons/nifi-utils/pom.xml index 86fd309bfd..347478637a 100644 --- a/nifi/nifi-commons/nifi-utils/pom.xml +++ b/nifi/nifi-commons/nifi-utils/pom.xml @@ -18,10 +18,10 @@ org.apache.nifi nifi-commons - 0.0.3-incubating-SNAPSHOT + 0.1.0-incubating-SNAPSHOT nifi-utils - 0.0.3-incubating-SNAPSHOT + 0.1.0-incubating-SNAPSHOT jar org.apache.nifi nifi-api - 0.0.3-incubating-SNAPSHOT + 0.1.0-incubating-SNAPSHOT provided org.apache.nifi nifi-runtime - 0.0.3-incubating-SNAPSHOT + 0.1.0-incubating-SNAPSHOT provided org.apache.nifi nifi-nar-utils - 0.0.3-incubating-SNAPSHOT + 0.1.0-incubating-SNAPSHOT provided org.apache.nifi nifi-properties - 0.0.3-incubating-SNAPSHOT + 0.1.0-incubating-SNAPSHOT provided diff --git a/nifi/pom.xml b/nifi/pom.xml index 8ba4108c9d..8547257972 100644 --- a/nifi/pom.xml +++ b/nifi/pom.xml @@ -23,7 +23,7 @@ org.apache.nifi nifi - 0.0.3-incubating-SNAPSHOT + 0.1.0-incubating-SNAPSHOT pom Apache NiFi(incubating) is an easy to use, powerful, and reliable system to process and distribute data. http://nifi.incubator.apache.org @@ -647,62 +647,62 @@ org.apache.nifi nifi-api - 0.0.3-incubating-SNAPSHOT + 0.1.0-incubating-SNAPSHOT org.apache.nifi nifi-utils - 0.0.3-incubating-SNAPSHOT + 0.1.0-incubating-SNAPSHOT org.apache.nifi nifi-site-to-site-client - 0.0.3-incubating-SNAPSHOT + 0.1.0-incubating-SNAPSHOT org.apache.nifi nifi-web-utils - 0.0.3-incubating-SNAPSHOT + 0.1.0-incubating-SNAPSHOT org.apache.nifi nifi-expression-language - 0.0.3-incubating-SNAPSHOT + 0.1.0-incubating-SNAPSHOT org.apache.nifi nifi-custom-ui-utilities - 0.0.3-incubating-SNAPSHOT + 0.1.0-incubating-SNAPSHOT org.apache.nifi nifi-flowfile-packager - 0.0.3-incubating-SNAPSHOT + 0.1.0-incubating-SNAPSHOT org.apache.nifi nifi-socket-utils - 0.0.3-incubating-SNAPSHOT + 0.1.0-incubating-SNAPSHOT org.apache.nifi nifi-data-provenance-utils - 0.0.3-incubating-SNAPSHOT + 0.1.0-incubating-SNAPSHOT org.apache.nifi nifi-runtime - 0.0.3-incubating-SNAPSHOT + 0.1.0-incubating-SNAPSHOT org.apache.nifi nifi-bootstrap - 0.0.3-incubating-SNAPSHOT + 0.1.0-incubating-SNAPSHOT org.apache.nifi nifi-resources - 0.0.3-incubating-SNAPSHOT + 0.1.0-incubating-SNAPSHOT resources runtime zip @@ -710,7 +710,7 @@ org.apache.nifi nifi-docs - 0.0.3-incubating-SNAPSHOT + 0.1.0-incubating-SNAPSHOT resources runtime zip @@ -718,116 +718,116 @@ org.apache.nifi nifi-framework-nar - 0.0.3-incubating-SNAPSHOT + 0.1.0-incubating-SNAPSHOT nar org.apache.nifi nifi-provenance-repository-nar - 0.0.3-incubating-SNAPSHOT + 0.1.0-incubating-SNAPSHOT nar org.apache.nifi nifi-standard-services-api-nar - 0.0.3-incubating-SNAPSHOT + 0.1.0-incubating-SNAPSHOT nar org.apache.nifi nifi-ssl-context-service-nar - 0.0.3-incubating-SNAPSHOT + 0.1.0-incubating-SNAPSHOT nar org.apache.nifi nifi-distributed-cache-services-nar - 0.0.3-incubating-SNAPSHOT + 0.1.0-incubating-SNAPSHOT nar org.apache.nifi nifi-standard-nar - 0.0.3-incubating-SNAPSHOT + 0.1.0-incubating-SNAPSHOT nar org.apache.nifi nifi-jetty-bundle - 0.0.3-incubating-SNAPSHOT + 0.1.0-incubating-SNAPSHOT nar org.apache.nifi nifi-update-attribute-nar - 0.0.3-incubating-SNAPSHOT + 0.1.0-incubating-SNAPSHOT nar org.apache.nifi nifi-hadoop-libraries-nar - 0.0.3-incubating-SNAPSHOT + 0.1.0-incubating-SNAPSHOT nar org.apache.nifi nifi-hadoop-nar - 0.0.3-incubating-SNAPSHOT + 0.1.0-incubating-SNAPSHOT nar org.apache.nifi nifi-kite-nar - 0.0.3-incubating-SNAPSHOT + 0.1.0-incubating-SNAPSHOT nar org.apache.nifi nifi-kafka-nar - 0.0.3-incubating-SNAPSHOT + 0.1.0-incubating-SNAPSHOT nar org.apache.nifi nifi-http-context-map-nar - 0.0.3-incubating-SNAPSHOT + 0.1.0-incubating-SNAPSHOT nar org.apache.nifi nifi-properties - 0.0.3-incubating-SNAPSHOT + 0.1.0-incubating-SNAPSHOT org.apache.nifi nifi-security-utils - 0.0.3-incubating-SNAPSHOT + 0.1.0-incubating-SNAPSHOT org.apache.nifi nifi-logging-utils - 0.0.3-incubating-SNAPSHOT + 0.1.0-incubating-SNAPSHOT org.apache.nifi nifi-nar-utils - 0.0.3-incubating-SNAPSHOT + 0.1.0-incubating-SNAPSHOT org.apache.nifi nifi-processor-utils - 0.0.3-incubating-SNAPSHOT + 0.1.0-incubating-SNAPSHOT org.apache.nifi nifi-mock - 0.0.3-incubating-SNAPSHOT + 0.1.0-incubating-SNAPSHOT test org.apache.nifi nifi-write-ahead-log - 0.0.3-incubating-SNAPSHOT + 0.1.0-incubating-SNAPSHOT com.jayway.jsonpath From fe64d82c5810e4c4d278ae101c2ce18ae5adddfa Mon Sep 17 00:00:00 2001 From: joewitt Date: Tue, 17 Mar 2015 12:37:39 -0400 Subject: [PATCH 23/26] NIFI-428 This closes #36 --- .../nifi-kite-bundle/nifi-kite-nar/pom.xml | 42 ++++++++++++++++++- .../src/main/resources/META-INF/LICENSE | 38 ----------------- .../src/main/resources/META-INF/NOTICE | 37 +++------------- .../nifi-kite-processors/pom.xml | 14 ------- .../nifi-nar-bundles/nifi-kite-bundle/pom.xml | 14 ------- nifi/pom.xml | 1 + 6 files changed, 47 insertions(+), 99 deletions(-) diff --git a/nifi/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-nar/pom.xml b/nifi/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-nar/pom.xml index 13479e0be3..66d837c702 100644 --- a/nifi/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-nar/pom.xml +++ b/nifi/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-nar/pom.xml @@ -33,12 +33,52 @@ org.apache.nifi nifi-kite-processors - + org.apache.hadoop hadoop-client + + org.xerial.snappy + snappy-java + + + com.thoughtworks.paranamer + paranamer + + + org.codehaus.jackson + jackson-mapper-asl + + + org.codehaus.jackson + jackson-core-asl + + + com.google.guava + guava + + + com.google.guava + guava + + + commons-logging + commons-logging + + + org.apache.commons + commons-compress + + + commons-codec + commons-codec + + + org.apache.avro + avro + diff --git a/nifi/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-nar/src/main/resources/META-INF/LICENSE b/nifi/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-nar/src/main/resources/META-INF/LICENSE index 6fb2525c34..d645695673 100644 --- a/nifi/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-nar/src/main/resources/META-INF/LICENSE +++ b/nifi/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-nar/src/main/resources/META-INF/LICENSE @@ -200,41 +200,3 @@ 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. - -APACHE NIFI SUBCOMPONENTS: - -The Apache NiFi project contains subcomponents with separate copyright -notices and license terms. Your use of the source code for the these -subcomponents is subject to the terms and conditions of the following -licenses. - -This product bundles 'ParaNamer' which is available under a BSD style license. -For details see http://asm.ow2.org/asmdex-license.html - - Copyright (c) 2006 Paul Hammant & ThoughtWorks Inc - All rights reserved. - - Redistribution and use in source and binary forms, with or without - modification, are permitted provided that the following conditions - are met: - 1. Redistributions of source code must retain the above copyright - notice, this list of conditions and the following disclaimer. - 2. Redistributions in binary form must reproduce the above copyright - notice, this list of conditions and the following disclaimer in the - documentation and/or other materials provided with the distribution. - 3. Neither the name of the copyright holders nor the names of its - contributors may be used to endorse or promote products derived from - this software without specific prior written permission. - - THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" - AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE - IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE - ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE - LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR - CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF - SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS - INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN - CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) - ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF - THE POSSIBILITY OF SUCH DAMAGE. - diff --git a/nifi/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-nar/src/main/resources/META-INF/NOTICE b/nifi/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-nar/src/main/resources/META-INF/NOTICE index 66204cfa7f..330a9af666 100644 --- a/nifi/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-nar/src/main/resources/META-INF/NOTICE +++ b/nifi/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-nar/src/main/resources/META-INF/NOTICE @@ -15,33 +15,19 @@ The following binary components are provided under the Apache Software License v Apache Avro Copyright 2009-2013 The Apache Software Foundation - (ASLv2) Apache Commons Codec + (ASLv2) Apache Commons Lang The following NOTICE information applies: - Apache Commons Codec - Copyright 2002-2014 The Apache Software Foundation + Apache Commons Lang + Copyright 2001-2014 The Apache Software Foundation - src/test/org/apache/commons/codec/language/DoubleMetaphoneTest.java - contains test data from http://aspell.net/test/orig/batch0.tab. - Copyright (C) 2002 Kevin Atkinson (kevina@gnu.org) - - =============================================================================== - - The content of package org.apache.commons.codec.language.bm has been translated - from the original php source code available at http://stevemorse.org/phoneticinfo.htm - with permission from the original authors. - Original source copyright: - Copyright (c) 2008 Alexander Beider & Stephen P. Morse. + This product includes software from the Spring Framework, + under the Apache License 2.0 (see: StringUtils.containsWhitespace()) (ASLv2) Apache Commons JEXL The following NOTICE information applies: Apache Commons JEXL Copyright 2001-2011 The Apache Software Foundation - (ASLv2) Apache Commons Logging - The following NOTICE information applies: - Apache Commons Logging - Copyright 2003-2013 The Apache Software Foundation - (ASLv2) Kite SDK The following NOTICE information applies: This product includes software developed by Cloudera, Inc. @@ -53,19 +39,6 @@ The following binary components are provided under the Apache Software License v This product includes software developed by Saxonica (http://www.saxonica.com/). - (ASLv2) Snappy Java - The following NOTICE information applies: - This product includes software developed by Google - Snappy: http://code.google.com/p/snappy/ (New BSD License) - - This product includes software developed by Apache - PureJavaCrc32C from apache-hadoop-common http://hadoop.apache.org/ - (Apache 2.0 license) - - This library contains statically linked libstdc++. This inclusion is allowed by - "GCC RUntime Library Exception" - http://gcc.gnu.org/onlinedocs/libstdc++/manual/license.html - (ASLv2) Parquet MR The following NOTICE information applies: Parquet MR diff --git a/nifi/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-processors/pom.xml b/nifi/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-processors/pom.xml index b18aa7d95a..938854d3a1 100644 --- a/nifi/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-processors/pom.xml +++ b/nifi/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-processors/pom.xml @@ -26,8 +26,6 @@ 1.0.0 - 11.0.2 - 4.10 1.3.9-1 @@ -38,18 +36,10 @@ org.apache.nifi nifi-api - - org.apache.nifi - nifi-utils - org.apache.nifi nifi-processor-utils - - org.apache.nifi - nifi-flowfile-packager - @@ -86,8 +76,6 @@ com.google.guava guava - ${guava.version} - compile @@ -104,7 +92,6 @@ junit junit test - ${junit.version} @@ -123,7 +110,6 @@ com.sun.jersey jersey-servlet - 1.14 test diff --git a/nifi/nifi-nar-bundles/nifi-kite-bundle/pom.xml b/nifi/nifi-nar-bundles/nifi-kite-bundle/pom.xml index 6391aa983e..21e378d7a3 100644 --- a/nifi/nifi-nar-bundles/nifi-kite-bundle/pom.xml +++ b/nifi/nifi-nar-bundles/nifi-kite-bundle/pom.xml @@ -30,20 +30,6 @@ nifi-kite-processors nifi-kite-nar - - - - - - org.apache.maven.plugins - maven-surefire-plugin - - true - - - - - diff --git a/nifi/pom.xml b/nifi/pom.xml index 8547257972..e7d8f5ba5e 100644 --- a/nifi/pom.xml +++ b/nifi/pom.xml @@ -887,6 +887,7 @@ maven-surefire-plugin 2.18 + true -Xmx1G From 1cca300d1d71a78ba0921b60ef841e3036c1e5b8 Mon Sep 17 00:00:00 2001 From: joewitt Date: Tue, 17 Mar 2015 12:54:45 -0400 Subject: [PATCH 24/26] This closes #10 Feedback on the PR have been provided in NIFI-121 From e9d31f5d994e5d6d89edf8465942167647959abc Mon Sep 17 00:00:00 2001 From: joewitt Date: Tue, 17 Mar 2015 23:42:22 -0400 Subject: [PATCH 25/26] NIFI-280 This closes #19 Updated poms for versions after merge. Minor doc removals. Awesome PR! --- nifi/nifi-assembly/NOTICE | 5 + .../nifi-framework/nifi-documentation/pom.xml | 92 +-- .../ConfigurableComponentInitializer.java | 16 +- .../nifi/documentation/DocGenerator.java | 227 +++--- .../documentation/DocumentationWriter.java | 6 +- .../html/HtmlDocumentationWriter.java | 650 +++++++++--------- .../HtmlProcessorDocumentationWriter.java | 46 +- .../init/ControllerServiceInitializer.java | 12 +- .../init/ProcessorInitializer.java | 12 +- .../init/ReportingTaskingInitializer.java | 12 +- ...ontrollerServiceInitializationContext.java | 18 +- .../mock/MockControllerServiceLookup.java | 36 +- .../MockProcessorInitializationContext.java | 26 +- .../MockReportingInitializationContext.java | 50 +- .../documentation/example/NakedProcessor.java | 17 +- .../nifi-framework/pom.xml | 2 +- .../nifi-framework-bundle/pom.xml | 2 +- .../processors/standard/DistributeLoad.java | 2 +- .../standard/HandleHttpRequest.java | 7 +- .../additionalDetails.html | 42 +- .../additionalDetails.html | 58 ++ .../index.html | 155 ----- .../additionalDetails.html | 222 +++--- .../index.html | 100 --- .../additionalDetails.html | 127 ++++ .../index.html | 255 ------- .../additionalDetails.html | 57 ++ .../index.html | 112 --- .../additionalDetails.html | 88 +-- .../additionalDetails.html | 2 +- .../additionalDetails.html | 2 +- .../additionalDetails.html | 2 +- .../additionalDetails.html | 24 +- .../additionalDetails.html | 4 +- .../additionalDetails.html | 2 +- .../additionalDetails.html | 2 +- .../additionalDetails.html | 2 +- .../additionalDetails.html | 2 +- .../index.html | 85 --- 39 files changed, 1008 insertions(+), 1573 deletions(-) create mode 100644 nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.EvaluateJsonPath/additionalDetails.html delete mode 100644 nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.EvaluateJsonPath/index.html delete mode 100644 nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.ExecuteProcess/index.html create mode 100644 nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.HandleHttpRequest/additionalDetails.html delete mode 100644 nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.HandleHttpRequest/index.html create mode 100644 nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.HandleHttpResponse/additionalDetails.html delete mode 100644 nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.HandleHttpResponse/index.html delete mode 100644 nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.SplitJson/index.html diff --git a/nifi/nifi-assembly/NOTICE b/nifi/nifi-assembly/NOTICE index 4ead623d50..8d7db8d7ec 100644 --- a/nifi/nifi-assembly/NOTICE +++ b/nifi/nifi-assembly/NOTICE @@ -181,6 +181,11 @@ The following binary components are provided under the Apache Software License v Apache log4j Copyright 2007 The Apache Software Foundation + (ASLv2) Apache Tika + The following NOTICE information applies: + Apache Tika Core + Copyright 2007-2015 The Apache Software Foundation + (ASLv2) Apache Commons Configuration The following NOTICE information applies: Apache Commons Configuration diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/pom.xml b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/pom.xml index f35c8cd83c..e522d30ff7 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/pom.xml +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/pom.xml @@ -1,49 +1,49 @@ - - 4.0.0 - - org.apache.nifi - nifi-framework - 0.0.2-incubating-SNAPSHOT - - nifi-documentation - - - org.apache.nifi - nifi-nar-utils - - - org.apache.nifi - nifi-api - - - org.apache.nifi - nifi-properties - - - commons-io - commons-io - - - org.apache.commons - commons-lang3 - - - org.apache.nifi - nifi-processor-utils - test - - + xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd" + xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"> + + 4.0.0 + + org.apache.nifi + nifi-framework + 0.1.0-incubating-SNAPSHOT + + nifi-documentation + + + org.apache.nifi + nifi-nar-utils + + + org.apache.nifi + nifi-api + + + org.apache.nifi + nifi-properties + + + commons-io + commons-io + + + org.apache.commons + commons-lang3 + + + org.apache.nifi + nifi-processor-utils + test + + diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/ConfigurableComponentInitializer.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/ConfigurableComponentInitializer.java index 164d212bd0..bd07ab54d8 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/ConfigurableComponentInitializer.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/ConfigurableComponentInitializer.java @@ -27,12 +27,12 @@ import org.apache.nifi.reporting.InitializationException; */ 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; + /** + * 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; } diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/DocGenerator.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/DocGenerator.java index 157d95edf5..8a53f00ca0 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/DocGenerator.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/DocGenerator.java @@ -43,140 +43,137 @@ import org.slf4j.LoggerFactory; /** * Uses the ExtensionManager to get a list of Processor, ControllerService, and * Reporting Task classes that were loaded and generate documentation for them. - * + * * */ public class DocGenerator { - private static final Logger logger = LoggerFactory.getLogger(DocGenerator.class); + private static final Logger logger = LoggerFactory.getLogger(DocGenerator.class); - /** - * Generates documentation into the work/docs dir specified by - * NiFiProperties. - * - * @param properties - */ - public static void generate(final NiFiProperties properties) { - @SuppressWarnings("rawtypes") - final Set extensionClasses = new HashSet<>(); - extensionClasses.addAll(ExtensionManager.getExtensions(Processor.class)); - extensionClasses.addAll(ExtensionManager.getExtensions(ControllerService.class)); - extensionClasses.addAll(ExtensionManager.getExtensions(ReportingTask.class)); + /** + * Generates documentation into the work/docs dir specified by + * NiFiProperties. + * + * @param properties + */ + public static void generate(final NiFiProperties properties) { + @SuppressWarnings("rawtypes") + final Set extensionClasses = new HashSet<>(); + extensionClasses.addAll(ExtensionManager.getExtensions(Processor.class)); + extensionClasses.addAll(ExtensionManager.getExtensions(ControllerService.class)); + extensionClasses.addAll(ExtensionManager.getExtensions(ReportingTask.class)); - final File explodedNiFiDocsDir = properties.getComponentDocumentationWorkingDirectory(); + final File explodedNiFiDocsDir = properties.getComponentDocumentationWorkingDirectory(); - logger.debug("Generating documentation for: " + extensionClasses.size() + " components in: " - + explodedNiFiDocsDir); + logger.debug("Generating documentation for: " + extensionClasses.size() + " components in: " + + explodedNiFiDocsDir); - for (final Class extensionClass : extensionClasses) { - if (ConfigurableComponent.class.isAssignableFrom(extensionClass)) { - final Class componentClass = extensionClass - .asSubclass(ConfigurableComponent.class); - try { - logger.debug("Documenting: " + componentClass); - document(explodedNiFiDocsDir, componentClass); - } catch (Exception e) { - logger.warn("Unable to document: " + componentClass); - } - } - } - } + for (final Class extensionClass : extensionClasses) { + if (ConfigurableComponent.class.isAssignableFrom(extensionClass)) { + final Class componentClass = extensionClass + .asSubclass(ConfigurableComponent.class); + try { + logger.debug("Documenting: " + componentClass); + document(explodedNiFiDocsDir, componentClass); + } catch (Exception e) { + logger.warn("Unable to document: " + componentClass); + } + } + } + } - /** - * Generates the documentation for a particular configurable comopnent. Will - * check to see if an "additionalDetails.html" file exists and will link - * that from the generated documentation. - * - * @param docsDir - * the work\docs\components dir to stick component documentation - * in - * @param componentClass - * the class to document - * @throws InstantiationException - * @throws IllegalAccessException - * @throws IOException - * @throws InitializationException - */ - private static void document(final File docsDir, final Class componentClass) - throws InstantiationException, IllegalAccessException, IOException, InitializationException { + /** + * Generates the documentation for a particular configurable comopnent. Will + * check to see if an "additionalDetails.html" file exists and will link + * that from the generated documentation. + * + * @param docsDir the work\docs\components dir to stick component + * documentation in + * @param componentClass the class to document + * @throws InstantiationException + * @throws IllegalAccessException + * @throws IOException + * @throws InitializationException + */ + private static void document(final File docsDir, final Class componentClass) + throws InstantiationException, IllegalAccessException, IOException, InitializationException { - final ConfigurableComponent component = componentClass.newInstance(); - final ConfigurableComponentInitializer initializer = getComponentInitializer(componentClass); - initializer.initialize(component); + final ConfigurableComponent component = componentClass.newInstance(); + final ConfigurableComponentInitializer initializer = getComponentInitializer(componentClass); + initializer.initialize(component); - final DocumentationWriter writer = getDocumentWriter(componentClass); + final DocumentationWriter writer = getDocumentWriter(componentClass); - final File directory = new File(docsDir, componentClass.getCanonicalName()); - directory.mkdirs(); + final File directory = new File(docsDir, componentClass.getCanonicalName()); + directory.mkdirs(); - final File baseDocumenationFile = new File(directory, "index.html"); - if (baseDocumenationFile.exists()) { - logger.warn(baseDocumenationFile + " already exists, overwriting!"); - } + final File baseDocumenationFile = new File(directory, "index.html"); + if (baseDocumenationFile.exists()) { + logger.warn(baseDocumenationFile + " already exists, overwriting!"); + } - try (final OutputStream output = new BufferedOutputStream(new FileOutputStream(baseDocumenationFile))) { - writer.write(component, output, hasAdditionalInfo(directory)); - } - } + try (final OutputStream output = new BufferedOutputStream(new FileOutputStream(baseDocumenationFile))) { + writer.write(component, output, hasAdditionalInfo(directory)); + } + } - /** - * Returns the DocumentationWriter for the type of component. Currently - * Processor, ControllerService, and ReportingTask are supported. - * - * @param componentClass - * the class that requires a DocumentationWriter - * @return a DocumentationWriter capable of generating documentation for - * that specific type of class - */ - private static DocumentationWriter getDocumentWriter(final Class componentClass) { - if (Processor.class.isAssignableFrom(componentClass)) { - return new HtmlProcessorDocumentationWriter(); - } else if (ControllerService.class.isAssignableFrom(componentClass)) { - return new HtmlDocumentationWriter(); - } else if (ReportingTask.class.isAssignableFrom(componentClass)) { - return new HtmlDocumentationWriter(); - } + /** + * Returns the DocumentationWriter for the type of component. Currently + * Processor, ControllerService, and ReportingTask are supported. + * + * @param componentClass the class that requires a DocumentationWriter + * @return a DocumentationWriter capable of generating documentation for + * that specific type of class + */ + private static DocumentationWriter getDocumentWriter(final Class componentClass) { + if (Processor.class.isAssignableFrom(componentClass)) { + return new HtmlProcessorDocumentationWriter(); + } else if (ControllerService.class.isAssignableFrom(componentClass)) { + return new HtmlDocumentationWriter(); + } else if (ReportingTask.class.isAssignableFrom(componentClass)) { + return new HtmlDocumentationWriter(); + } - return null; - } + return null; + } - /** - * 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 - */ - private static ConfigurableComponentInitializer getComponentInitializer( - final Class 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(); - } + /** + * 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 + */ + private static ConfigurableComponentInitializer getComponentInitializer( + final Class 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; - } + return null; + } - /** - * Checks to see if a directory to write to has an additionalDetails.html in - * it already. - * - * @param directory - * @return true if additionalDetails.html exists, false otherwise. - */ - private static boolean hasAdditionalInfo(File directory) { - return directory.list(new FilenameFilter() { + /** + * Checks to see if a directory to write to has an additionalDetails.html in + * it already. + * + * @param directory + * @return true if additionalDetails.html exists, false otherwise. + */ + private static boolean hasAdditionalInfo(File directory) { + return directory.list(new FilenameFilter() { - @Override - public boolean accept(File dir, String name) { - return name.equalsIgnoreCase(HtmlDocumentationWriter.ADDITIONAL_DETAILS_HTML); - } + @Override + public boolean accept(File dir, String name) { + return name.equalsIgnoreCase(HtmlDocumentationWriter.ADDITIONAL_DETAILS_HTML); + } - }).length > 0; - } + }).length > 0; + } } diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/DocumentationWriter.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/DocumentationWriter.java index 5a3c5d8a80..d17863651a 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/DocumentationWriter.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/DocumentationWriter.java @@ -23,11 +23,11 @@ import org.apache.nifi.components.ConfigurableComponent; /** * Generates documentation for an instance of a ConfigurableComponent - * + * * */ public interface DocumentationWriter { - void write(ConfigurableComponent configurableComponent, OutputStream streamToWriteTo, - boolean includesAdditionalDocumentation) throws IOException; + void write(ConfigurableComponent configurableComponent, OutputStream streamToWriteTo, + boolean includesAdditionalDocumentation) throws IOException; } diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/html/HtmlDocumentationWriter.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/html/HtmlDocumentationWriter.java index c192759b12..b547934d0d 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/html/HtmlDocumentationWriter.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/html/HtmlDocumentationWriter.java @@ -38,380 +38,354 @@ import org.apache.nifi.documentation.DocumentationWriter; * Generates HTML documentation for a ConfigurableComponent. This class is used * to generate documentation for ControllerService and ReportingTask because * they have no additional information. - * + * * */ public class HtmlDocumentationWriter implements DocumentationWriter { - /** - * The filename where additional user specified information may be stored. - */ - public static final String ADDITIONAL_DETAILS_HTML = "additionalDetails.html"; + /** + * The filename where additional user specified information may be stored. + */ + public static final String ADDITIONAL_DETAILS_HTML = "additionalDetails.html"; - @Override - public void write(final ConfigurableComponent configurableComponent, final OutputStream streamToWriteTo, - final boolean includesAdditionalDocumentation) throws IOException { + @Override + public void write(final ConfigurableComponent configurableComponent, final OutputStream streamToWriteTo, + final boolean includesAdditionalDocumentation) throws IOException { - try { - XMLStreamWriter xmlStreamWriter = XMLOutputFactory.newInstance().createXMLStreamWriter( - streamToWriteTo, "UTF-8"); - xmlStreamWriter.writeDTD(""); - xmlStreamWriter.writeStartElement("html"); - xmlStreamWriter.writeAttribute("lang", "en"); - writeHead(configurableComponent, xmlStreamWriter); - writeBody(configurableComponent, xmlStreamWriter, includesAdditionalDocumentation); - xmlStreamWriter.writeEndElement(); - xmlStreamWriter.close(); - } catch (XMLStreamException | FactoryConfigurationError e) { - throw new IOException("Unable to create XMLOutputStream", e); - } - } + try { + XMLStreamWriter xmlStreamWriter = XMLOutputFactory.newInstance().createXMLStreamWriter( + streamToWriteTo, "UTF-8"); + xmlStreamWriter.writeDTD(""); + xmlStreamWriter.writeStartElement("html"); + xmlStreamWriter.writeAttribute("lang", "en"); + writeHead(configurableComponent, xmlStreamWriter); + writeBody(configurableComponent, xmlStreamWriter, includesAdditionalDocumentation); + xmlStreamWriter.writeEndElement(); + xmlStreamWriter.close(); + } catch (XMLStreamException | FactoryConfigurationError e) { + throw new IOException("Unable to create XMLOutputStream", e); + } + } - /** - * Writes the head portion of the HTML documentation. - * - * @param configurableComponent - * the component to describe - * @param xmlStreamWriter - * the stream to write to - * @throws XMLStreamException - * thrown if there was a problem writing to the stream - */ - protected void writeHead(final ConfigurableComponent configurableComponent, - final XMLStreamWriter xmlStreamWriter) throws XMLStreamException { - xmlStreamWriter.writeStartElement("head"); - xmlStreamWriter.writeStartElement("meta"); - xmlStreamWriter.writeAttribute("charset", "utf-8"); - xmlStreamWriter.writeEndElement(); - writeSimpleElement(xmlStreamWriter, "title", getTitle(configurableComponent)); + /** + * Writes the head portion of the HTML documentation. + * + * @param configurableComponent the component to describe + * @param xmlStreamWriter the stream to write to + * @throws XMLStreamException thrown if there was a problem writing to the + * stream + */ + protected void writeHead(final ConfigurableComponent configurableComponent, + final XMLStreamWriter xmlStreamWriter) throws XMLStreamException { + xmlStreamWriter.writeStartElement("head"); + xmlStreamWriter.writeStartElement("meta"); + xmlStreamWriter.writeAttribute("charset", "utf-8"); + xmlStreamWriter.writeEndElement(); + writeSimpleElement(xmlStreamWriter, "title", getTitle(configurableComponent)); - xmlStreamWriter.writeStartElement("link"); - xmlStreamWriter.writeAttribute("rel", "stylesheet"); - xmlStreamWriter.writeAttribute("href", "../../css/component-usage.css"); - xmlStreamWriter.writeAttribute("type", "text/css"); - xmlStreamWriter.writeEndElement(); + xmlStreamWriter.writeStartElement("link"); + xmlStreamWriter.writeAttribute("rel", "stylesheet"); + xmlStreamWriter.writeAttribute("href", "../../css/component-usage.css"); + xmlStreamWriter.writeAttribute("type", "text/css"); + xmlStreamWriter.writeEndElement(); - xmlStreamWriter.writeEndElement(); - } + xmlStreamWriter.writeEndElement(); + } - /** - * Gets the class name of the component. - * - * @param configurableComponent - * the component to describe - * @return the class name of the component - */ - protected String getTitle(final ConfigurableComponent configurableComponent) { - return configurableComponent.getClass().getSimpleName(); - } + /** + * Gets the class name of the component. + * + * @param configurableComponent the component to describe + * @return the class name of the component + */ + protected String getTitle(final ConfigurableComponent configurableComponent) { + return configurableComponent.getClass().getSimpleName(); + } - /** - * Writes the body section of the documentation, this consists of the - * component description, the tags, and the PropertyDescriptors. - * - * @param configurableComponent - * the component to describe - * @param xmlStreamWriter - * the stream writer - * @param hasAdditionalDetails - * whether there are additional details present or not - * @throws XMLStreamException - * thrown if there was a problem writing to the XML stream - */ - private final void writeBody(final ConfigurableComponent configurableComponent, - final XMLStreamWriter xmlStreamWriter, final boolean hasAdditionalDetails) - throws XMLStreamException { - xmlStreamWriter.writeStartElement("body"); - writeDescription(configurableComponent, xmlStreamWriter, hasAdditionalDetails); - writeTags(configurableComponent, xmlStreamWriter); - writeProperties(configurableComponent, xmlStreamWriter); - writeAdditionalBodyInfo(configurableComponent, xmlStreamWriter); - xmlStreamWriter.writeEndElement(); - } + /** + * Writes the body section of the documentation, this consists of the + * component description, the tags, and the PropertyDescriptors. + * + * @param configurableComponent the component to describe + * @param xmlStreamWriter the stream writer + * @param hasAdditionalDetails whether there are additional details present + * or not + * @throws XMLStreamException thrown if there was a problem writing to the + * XML stream + */ + private void writeBody(final ConfigurableComponent configurableComponent, + final XMLStreamWriter xmlStreamWriter, final boolean hasAdditionalDetails) + throws XMLStreamException { + xmlStreamWriter.writeStartElement("body"); + writeDescription(configurableComponent, xmlStreamWriter, hasAdditionalDetails); + writeTags(configurableComponent, xmlStreamWriter); + writeProperties(configurableComponent, xmlStreamWriter); + writeAdditionalBodyInfo(configurableComponent, xmlStreamWriter); + xmlStreamWriter.writeEndElement(); + } - /** - * This method may be overridden by sub classes to write additional - * information to the body of the documentation. - * - * @param configurableComponent - * the component to describe - * @param xmlStreamWriter - * the stream writer - * @throws XMLStreamException - * thrown if there was a problem writing to the XML stream - */ - protected void writeAdditionalBodyInfo(final ConfigurableComponent configurableComponent, - final XMLStreamWriter xmlStreamWriter) throws XMLStreamException { - } + /** + * This method may be overridden by sub classes to write additional + * information to the body of the documentation. + * + * @param configurableComponent the component to describe + * @param xmlStreamWriter the stream writer + * @throws XMLStreamException thrown if there was a problem writing to the + * XML stream + */ + protected void writeAdditionalBodyInfo(final ConfigurableComponent configurableComponent, + final XMLStreamWriter xmlStreamWriter) throws XMLStreamException { + } - /** - * Writes the tags attached to a ConfigurableComponent. - * - * @param configurableComponent - * @param xmlStreamWriter - * @throws XMLStreamException - */ - private void writeTags(final ConfigurableComponent configurableComponent, - final XMLStreamWriter xmlStreamWriter) throws XMLStreamException { - final Tags tags = configurableComponent.getClass().getAnnotation(Tags.class); - xmlStreamWriter.writeStartElement("h3"); - xmlStreamWriter.writeCharacters("Tags: "); - xmlStreamWriter.writeEndElement(); - xmlStreamWriter.writeStartElement("p"); - if (tags != null) { - final String tagString = StringUtils.join(tags.value(), ", "); - xmlStreamWriter.writeCharacters(tagString); - } else { - xmlStreamWriter.writeCharacters("None."); - } - xmlStreamWriter.writeEndElement(); + /** + * Writes the tags attached to a ConfigurableComponent. + * + * @param configurableComponent + * @param xmlStreamWriter + * @throws XMLStreamException + */ + private void writeTags(final ConfigurableComponent configurableComponent, + final XMLStreamWriter xmlStreamWriter) throws XMLStreamException { + final Tags tags = configurableComponent.getClass().getAnnotation(Tags.class); + xmlStreamWriter.writeStartElement("h3"); + xmlStreamWriter.writeCharacters("Tags: "); + xmlStreamWriter.writeEndElement(); + xmlStreamWriter.writeStartElement("p"); + if (tags != null) { + final String tagString = StringUtils.join(tags.value(), ", "); + xmlStreamWriter.writeCharacters(tagString); + } else { + xmlStreamWriter.writeCharacters("None."); + } + xmlStreamWriter.writeEndElement(); - } + } - /** - * Writes a description of the configurable component. - * - * @param configurableComponent - * the component to describe - * @param xmlStreamWriter - * the stream writer - * @param hasAdditionalDetails - * whether there are additional details available as - * 'additionalDetails.html' - * @throws XMLStreamException - * thrown if there was a problem writing to the XML stream - */ - protected void writeDescription(final ConfigurableComponent configurableComponent, - final XMLStreamWriter xmlStreamWriter, final boolean hasAdditionalDetails) - throws XMLStreamException { - writeSimpleElement(xmlStreamWriter, "h2", "Description: "); - writeSimpleElement(xmlStreamWriter, "p", getDescription(configurableComponent)); - if (hasAdditionalDetails) { - xmlStreamWriter.writeStartElement("p"); + /** + * Writes a description of the configurable component. + * + * @param configurableComponent the component to describe + * @param xmlStreamWriter the stream writer + * @param hasAdditionalDetails whether there are additional details + * available as 'additionalDetails.html' + * @throws XMLStreamException thrown if there was a problem writing to the + * XML stream + */ + protected void writeDescription(final ConfigurableComponent configurableComponent, + final XMLStreamWriter xmlStreamWriter, final boolean hasAdditionalDetails) + throws XMLStreamException { + writeSimpleElement(xmlStreamWriter, "h2", "Description: "); + writeSimpleElement(xmlStreamWriter, "p", getDescription(configurableComponent)); + if (hasAdditionalDetails) { + xmlStreamWriter.writeStartElement("p"); - writeLink(xmlStreamWriter, "Additional Details...", ADDITIONAL_DETAILS_HTML); + writeLink(xmlStreamWriter, "Additional Details...", ADDITIONAL_DETAILS_HTML); - xmlStreamWriter.writeEndElement(); - } - } + xmlStreamWriter.writeEndElement(); + } + } - /** - * Gets a description of the ConfigurableComponent using the - * CapabilityDescription annotation. - * - * @param configurableComponent - * the component to describe - * @return a description of the configurableComponent - */ - protected String getDescription(final ConfigurableComponent configurableComponent) { - final CapabilityDescription capabilityDescription = configurableComponent.getClass().getAnnotation( - CapabilityDescription.class); + /** + * Gets a description of the ConfigurableComponent using the + * CapabilityDescription annotation. + * + * @param configurableComponent the component to describe + * @return a description of the configurableComponent + */ + protected String getDescription(final ConfigurableComponent configurableComponent) { + final CapabilityDescription capabilityDescription = configurableComponent.getClass().getAnnotation( + CapabilityDescription.class); - final String description; - if (capabilityDescription != null) { - description = capabilityDescription.value(); - } else { - description = "No description provided."; - } + final String description; + if (capabilityDescription != null) { + description = capabilityDescription.value(); + } else { + description = "No description provided."; + } - return description; - } + return description; + } - /** - * Writes the PropertyDescriptors out as a table. - * - * @param configurableComponent - * the component to describe - * @param xmlStreamWriter - * the stream writer - * @throws XMLStreamException - * thrown if there was a problem writing to the XML Stream - */ - protected void writeProperties(final ConfigurableComponent configurableComponent, - final XMLStreamWriter xmlStreamWriter) throws XMLStreamException { - writeSimpleElement(xmlStreamWriter, "h3", "Properties: "); - xmlStreamWriter.writeStartElement("p"); - xmlStreamWriter.writeCharacters("In the list below, the names of required properties appear in "); - writeSimpleElement(xmlStreamWriter, "strong", "bold"); - xmlStreamWriter.writeCharacters(". Any " - + "other properties (not in bold) are considered optional. The table also " - + "indicates any default values, whether a property supports the "); - writeLink(xmlStreamWriter, "NiFi Expression Language", - "../../html/expression-language-guide.html"); - xmlStreamWriter.writeCharacters(", and whether a property is considered " - + "\"sensitive\", meaning that its value will be encrypted. Before entering a " - + "value in a sensitive property, ensure that the "); - writeSimpleElement(xmlStreamWriter, "strong", "nifi.properties"); - xmlStreamWriter.writeCharacters(" file has " + "an entry for the property "); - writeSimpleElement(xmlStreamWriter, "strong", "nifi.sensitive.props.key"); - xmlStreamWriter.writeCharacters("."); - xmlStreamWriter.writeEndElement(); + /** + * Writes the PropertyDescriptors out as a table. + * + * @param configurableComponent the component to describe + * @param xmlStreamWriter the stream writer + * @throws XMLStreamException thrown if there was a problem writing to the + * XML Stream + */ + protected void writeProperties(final ConfigurableComponent configurableComponent, + final XMLStreamWriter xmlStreamWriter) throws XMLStreamException { + writeSimpleElement(xmlStreamWriter, "h3", "Properties: "); + xmlStreamWriter.writeStartElement("p"); + xmlStreamWriter.writeCharacters("In the list below, the names of required properties appear in "); + writeSimpleElement(xmlStreamWriter, "strong", "bold"); + xmlStreamWriter.writeCharacters(". Any " + + "other properties (not in bold) are considered optional. The table also " + + "indicates any default values, whether a property supports the "); + writeLink(xmlStreamWriter, "NiFi Expression Language", + "../../html/expression-language-guide.html"); + xmlStreamWriter.writeCharacters(", and whether a property is considered " + + "\"sensitive\", meaning that its value will be encrypted. Before entering a " + + "value in a sensitive property, ensure that the "); + writeSimpleElement(xmlStreamWriter, "strong", "nifi.properties"); + xmlStreamWriter.writeCharacters(" file has " + "an entry for the property "); + writeSimpleElement(xmlStreamWriter, "strong", "nifi.sensitive.props.key"); + xmlStreamWriter.writeCharacters("."); + xmlStreamWriter.writeEndElement(); + List properties = configurableComponent.getPropertyDescriptors(); + if (properties.size() > 0) { + xmlStreamWriter.writeStartElement("table"); - List properties = configurableComponent.getPropertyDescriptors(); - if (properties.size() > 0) { - xmlStreamWriter.writeStartElement("table"); + // write the header row + xmlStreamWriter.writeStartElement("tr"); + writeSimpleElement(xmlStreamWriter, "th", "Name"); + writeSimpleElement(xmlStreamWriter, "th", "Default Value"); + writeSimpleElement(xmlStreamWriter, "th", "Valid Values"); + writeSimpleElement(xmlStreamWriter, "th", "Description"); + xmlStreamWriter.writeEndElement(); - // write the header row - xmlStreamWriter.writeStartElement("tr"); - writeSimpleElement(xmlStreamWriter, "th", "Name"); - writeSimpleElement(xmlStreamWriter, "th", "Default Value"); - writeSimpleElement(xmlStreamWriter, "th", "Valid Values"); - writeSimpleElement(xmlStreamWriter, "th", "Description"); - xmlStreamWriter.writeEndElement(); + // write the individual properties + for (PropertyDescriptor property : properties) { + xmlStreamWriter.writeStartElement("tr"); + xmlStreamWriter.writeStartElement("td"); + if (property.isRequired()) { + writeSimpleElement(xmlStreamWriter, "strong", property.getDisplayName()); + } else { + xmlStreamWriter.writeCharacters(property.getDisplayName()); + } - // write the individual properties - for (PropertyDescriptor property : properties) { - xmlStreamWriter.writeStartElement("tr"); - xmlStreamWriter.writeStartElement("td"); - if (property.isRequired()) { - writeSimpleElement(xmlStreamWriter, "strong", property.getDisplayName()); - } else { - xmlStreamWriter.writeCharacters(property.getDisplayName()); - } + xmlStreamWriter.writeEndElement(); + writeSimpleElement(xmlStreamWriter, "td", property.getDefaultValue()); + xmlStreamWriter.writeStartElement("td"); + writeValidValues(xmlStreamWriter, property); + xmlStreamWriter.writeEndElement(); + xmlStreamWriter.writeStartElement("td"); + if (property.getDescription() != null && property.getDescription().trim().length() > 0) { + xmlStreamWriter.writeCharacters(property.getDescription()); + } else { + xmlStreamWriter.writeCharacters("No Description Provided."); + } - xmlStreamWriter.writeEndElement(); - writeSimpleElement(xmlStreamWriter, "td", property.getDefaultValue()); - xmlStreamWriter.writeStartElement("td"); - writeValidValues(xmlStreamWriter, property); - xmlStreamWriter.writeEndElement(); - xmlStreamWriter.writeStartElement("td"); - if (property.getDescription() != null && property.getDescription().trim().length() > 0) { - xmlStreamWriter.writeCharacters(property.getDescription()); - } else { - xmlStreamWriter.writeCharacters("No Description Provided."); - } + if (property.isSensitive()) { + xmlStreamWriter.writeEmptyElement("br"); + writeSimpleElement(xmlStreamWriter, "strong", "Sensitive Property: true"); + } - if (property.isSensitive()) { - xmlStreamWriter.writeEmptyElement("br"); - writeSimpleElement(xmlStreamWriter, "strong", "Sensitive Property: true"); - } + if (property.isExpressionLanguageSupported()) { + xmlStreamWriter.writeEmptyElement("br"); + writeSimpleElement(xmlStreamWriter, "strong", "Supports Expression Language: true"); + } + xmlStreamWriter.writeEndElement(); - if (property.isExpressionLanguageSupported()) { - xmlStreamWriter.writeEmptyElement("br"); - writeSimpleElement(xmlStreamWriter, "strong", "Supports Expression Language: true"); - } - xmlStreamWriter.writeEndElement(); + xmlStreamWriter.writeEndElement(); + } - xmlStreamWriter.writeEndElement(); - } + // TODO support dynamic properties... + xmlStreamWriter.writeEndElement(); - // TODO support dynamic properties... - xmlStreamWriter.writeEndElement(); + } else { + writeSimpleElement(xmlStreamWriter, "p", "This component has no required or optional properties."); + } + } - } else { - writeSimpleElement(xmlStreamWriter, "p", "This component has no required or optional properties."); - } - } + private void writeValidValueDescription(XMLStreamWriter xmlStreamWriter, String description) + throws XMLStreamException { + xmlStreamWriter.writeCharacters(" "); + xmlStreamWriter.writeStartElement("img"); + xmlStreamWriter.writeAttribute("src", "../../html/images/iconInfo.png"); + xmlStreamWriter.writeAttribute("alt", description); + xmlStreamWriter.writeAttribute("title", description); + xmlStreamWriter.writeEndElement(); - private void writeValidValueDescription(XMLStreamWriter xmlStreamWriter, String description) - throws XMLStreamException { - xmlStreamWriter.writeCharacters(" "); - xmlStreamWriter.writeStartElement("img"); - xmlStreamWriter.writeAttribute("src", "../../html/images/iconInfo.png"); - xmlStreamWriter.writeAttribute("alt", description); - xmlStreamWriter.writeAttribute("title", description); - xmlStreamWriter.writeEndElement(); + } - } + /** + * Interrogates a PropertyDescriptor to get a list of AllowableValues, if + * there are none, nothing is written to the stream. + * + * @param xmlStreamWriter the stream writer to use + * @param property the property to describe + * @throws XMLStreamException thrown if there was a problem writing to the + * XML Stream + */ + protected void writeValidValues(XMLStreamWriter xmlStreamWriter, PropertyDescriptor property) + throws XMLStreamException { + if (property.getAllowableValues() != null && property.getAllowableValues().size() > 0) { + xmlStreamWriter.writeStartElement("ul"); + for (AllowableValue value : property.getAllowableValues()) { + xmlStreamWriter.writeStartElement("li"); + xmlStreamWriter.writeCharacters(value.getDisplayName()); - /** - * Interrogates a PropertyDescriptor to get a list of AllowableValues, if - * there are none, nothing is written to the stream. - * - * @param xmlStreamWriter - * the stream writer to use - * @param property - * the property to describe - * @throws XMLStreamException - * thrown if there was a problem writing to the XML Stream - */ - protected void writeValidValues(XMLStreamWriter xmlStreamWriter, PropertyDescriptor property) - throws XMLStreamException { - if (property.getAllowableValues() != null && property.getAllowableValues().size() > 0) { - xmlStreamWriter.writeStartElement("ul"); - for (AllowableValue value : property.getAllowableValues()) { - xmlStreamWriter.writeStartElement("li"); - xmlStreamWriter.writeCharacters(value.getDisplayName()); + if (value.getDescription() != null) { + writeValidValueDescription(xmlStreamWriter, value.getDescription()); + } + xmlStreamWriter.writeEndElement(); - if (value.getDescription() != null) { - writeValidValueDescription(xmlStreamWriter, value.getDescription()); - } - xmlStreamWriter.writeEndElement(); + } + xmlStreamWriter.writeEndElement(); + } else if (property.getControllerServiceDefinition() != null) { + Class controllerServiceClass = property.getControllerServiceDefinition(); + writeSimpleElement(xmlStreamWriter, "strong", "Controller Service: "); + xmlStreamWriter.writeEmptyElement("br"); + xmlStreamWriter.writeCharacters(controllerServiceClass.getSimpleName()); + } + } - } - xmlStreamWriter.writeEndElement(); - } else if (property.getControllerServiceDefinition() != null) { - Class controllerServiceClass = property.getControllerServiceDefinition(); - writeSimpleElement(xmlStreamWriter, "strong", "Controller Service: "); - xmlStreamWriter.writeEmptyElement("br"); - xmlStreamWriter.writeCharacters(controllerServiceClass.getSimpleName()); - } - } + /** + * Writes a begin element, then text, then end element for the element of a + * users choosing. Example: <p>text</p> + * + * @param writer the stream writer to use + * @param elementName the name of the element + * @param characters the characters to insert into the element + * @param strong whether the characters should be strong or not. + * @throws XMLStreamException thrown if there was a problem writing to the + * stream. + */ + protected final static void writeSimpleElement(final XMLStreamWriter writer, final String elementName, + final String characters, boolean strong) throws XMLStreamException { + writer.writeStartElement(elementName); + if (strong) { + writer.writeStartElement("strong"); + } + writer.writeCharacters(characters); + if (strong) { + writer.writeEndElement(); + } + writer.writeEndElement(); + } - /** - * Writes a begin element, then text, then end element for the element of a - * users choosing. Example: <p>text</p> - * - * @param writer - * the stream writer to use - * @param elementName - * the name of the element - * @param characters - * the characters to insert into the element - * @param strong - * whether the characters should be strong or not. - * @throws XMLStreamException - * thrown if there was a problem writing to the stream. - */ - protected final static void writeSimpleElement(final XMLStreamWriter writer, final String elementName, - final String characters, boolean strong) throws XMLStreamException { - writer.writeStartElement(elementName); - if (strong) { - writer.writeStartElement("strong"); - } - writer.writeCharacters(characters); - if (strong) { - writer.writeEndElement(); - } - writer.writeEndElement(); - } + /** + * Writes a begin element, then text, then end element for the element of a + * users choosing. Example: <p>text</p> + * + * @param writer the stream writer to use + * @param elementName the name of the element + * @param characters the characters to insert into the element + * @throws XMLStreamException thrown if there was a problem writing to the + * stream + */ + protected final static void writeSimpleElement(final XMLStreamWriter writer, final String elementName, + final String characters) throws XMLStreamException { + writeSimpleElement(writer, elementName, characters, false); + } - /** - * Writes a begin element, then text, then end element for the element of a - * users choosing. Example: <p>text</p> - * - * @param writer - * the stream writer to use - * @param elementName - * the name of the element - * @param characters - * the characters to insert into the element - * @throws XMLStreamException - * thrown if there was a problem writing to the stream - */ - protected final static void writeSimpleElement(final XMLStreamWriter writer, final String elementName, - final String characters) throws XMLStreamException { - writeSimpleElement(writer, elementName, characters, false); - } - - /** - * A helper method to write a link - * - * @param xmlStreamWriter - * the stream to write to - * @param text - * the text of the link - * @param location - * the location of the link - * @throws XMLStreamException - * thrown if there was a problem writing to the stream - */ - protected void writeLink(final XMLStreamWriter xmlStreamWriter, final String text, final String location) - throws XMLStreamException { - xmlStreamWriter.writeStartElement("a"); - xmlStreamWriter.writeAttribute("href", location); - xmlStreamWriter.writeCharacters(text); - xmlStreamWriter.writeEndElement(); - } + /** + * A helper method to write a link + * + * @param xmlStreamWriter the stream to write to + * @param text the text of the link + * @param location the location of the link + * @throws XMLStreamException thrown if there was a problem writing to the + * stream + */ + protected void writeLink(final XMLStreamWriter xmlStreamWriter, final String text, final String location) + throws XMLStreamException { + xmlStreamWriter.writeStartElement("a"); + xmlStreamWriter.writeAttribute("href", location); + xmlStreamWriter.writeCharacters(text); + xmlStreamWriter.writeEndElement(); + } } diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/html/HtmlProcessorDocumentationWriter.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/html/HtmlProcessorDocumentationWriter.java index 4cf6ba850e..412ebf6d43 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/html/HtmlProcessorDocumentationWriter.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/html/HtmlProcessorDocumentationWriter.java @@ -26,33 +26,33 @@ import org.apache.nifi.processor.Relationship; /** * Writes documentation specific for a Processor. This includes everything for a * ConfigurableComponent as well as Relationship information. - * + * * */ public class HtmlProcessorDocumentationWriter extends HtmlDocumentationWriter { - @Override - protected void writeAdditionalBodyInfo(final ConfigurableComponent configurableComponent, - final XMLStreamWriter xmlStreamWriter) throws XMLStreamException { - final Processor processor = (Processor) configurableComponent; - writeSimpleElement(xmlStreamWriter, "h3", "Relationships: "); + @Override + protected void writeAdditionalBodyInfo(final ConfigurableComponent configurableComponent, + final XMLStreamWriter xmlStreamWriter) throws XMLStreamException { + final Processor processor = (Processor) configurableComponent; + writeSimpleElement(xmlStreamWriter, "h3", "Relationships: "); - if (processor.getRelationships().size() > 0) { - xmlStreamWriter.writeStartElement("table"); - xmlStreamWriter.writeStartElement("tr"); - writeSimpleElement(xmlStreamWriter, "th", "Name"); - writeSimpleElement(xmlStreamWriter, "th", "Description"); - xmlStreamWriter.writeEndElement(); + if (processor.getRelationships().size() > 0) { + xmlStreamWriter.writeStartElement("table"); + xmlStreamWriter.writeStartElement("tr"); + writeSimpleElement(xmlStreamWriter, "th", "Name"); + writeSimpleElement(xmlStreamWriter, "th", "Description"); + xmlStreamWriter.writeEndElement(); - for (Relationship relationship : processor.getRelationships()) { - xmlStreamWriter.writeStartElement("tr"); - writeSimpleElement(xmlStreamWriter, "td", relationship.getName()); - writeSimpleElement(xmlStreamWriter, "td", relationship.getDescription()); - xmlStreamWriter.writeEndElement(); - } - xmlStreamWriter.writeEndElement(); - } else { - xmlStreamWriter.writeCharacters("This processor has no relationships."); - } - } + for (Relationship relationship : processor.getRelationships()) { + xmlStreamWriter.writeStartElement("tr"); + writeSimpleElement(xmlStreamWriter, "td", relationship.getName()); + writeSimpleElement(xmlStreamWriter, "td", relationship.getDescription()); + xmlStreamWriter.writeEndElement(); + } + xmlStreamWriter.writeEndElement(); + } else { + xmlStreamWriter.writeCharacters("This processor has no relationships."); + } + } } diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/init/ControllerServiceInitializer.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/init/ControllerServiceInitializer.java index 123a39c81d..3c66485923 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/init/ControllerServiceInitializer.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/init/ControllerServiceInitializer.java @@ -25,14 +25,14 @@ 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; - controllerService.initialize(new MockControllerServiceInitializationContext()); - } + @Override + public void initialize(ConfigurableComponent component) throws InitializationException { + ControllerService controllerService = (ControllerService) component; + controllerService.initialize(new MockControllerServiceInitializationContext()); + } } diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/init/ProcessorInitializer.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/init/ProcessorInitializer.java index a33f7b9acc..07e9c3ab8f 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/init/ProcessorInitializer.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/init/ProcessorInitializer.java @@ -23,15 +23,15 @@ import org.apache.nifi.processor.Processor; /** * Initializes a Procesor using a MockProcessorInitializationContext - * + * * */ public class ProcessorInitializer implements ConfigurableComponentInitializer { - @Override - public void initialize(ConfigurableComponent component) { - Processor processor = (Processor) component; - processor.initialize(new MockProcessorInitializationContext()); - } + @Override + public void initialize(ConfigurableComponent component) { + Processor processor = (Processor) component; + processor.initialize(new MockProcessorInitializationContext()); + } } diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/init/ReportingTaskingInitializer.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/init/ReportingTaskingInitializer.java index ff915cfa0b..6fcfca91da 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/init/ReportingTaskingInitializer.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/init/ReportingTaskingInitializer.java @@ -24,14 +24,14 @@ 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; - reportingTask.initialize(new MockReportingInitializationContext()); - } + @Override + public void initialize(ConfigurableComponent component) throws InitializationException { + ReportingTask reportingTask = (ReportingTask) component; + reportingTask.initialize(new MockReportingInitializationContext()); + } } diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/mock/MockControllerServiceInitializationContext.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/mock/MockControllerServiceInitializationContext.java index 6153a8effe..fcd3ea340f 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/mock/MockControllerServiceInitializationContext.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/mock/MockControllerServiceInitializationContext.java @@ -22,19 +22,19 @@ import org.apache.nifi.controller.ControllerServiceLookup; /** * 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 ""; - } + @Override + public String getIdentifier() { + return ""; + } - @Override - public ControllerServiceLookup getControllerServiceLookup() { - return new MockControllerServiceLookup(); - } + @Override + public ControllerServiceLookup getControllerServiceLookup() { + return new MockControllerServiceLookup(); + } } diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/mock/MockControllerServiceLookup.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/mock/MockControllerServiceLookup.java index 88d091155e..f11bc6874e 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/mock/MockControllerServiceLookup.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/mock/MockControllerServiceLookup.java @@ -26,30 +26,30 @@ import org.apache.nifi.controller.ControllerServiceLookup; * 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(String serviceIdentifier) { - return null; - } + @Override + public ControllerService getControllerService(String serviceIdentifier) { + return null; + } - @Override - public boolean isControllerServiceEnabled(String serviceIdentifier) { - return false; - } + @Override + public boolean isControllerServiceEnabled(String serviceIdentifier) { + return false; + } - @Override - public boolean isControllerServiceEnabled(ControllerService service) { - return false; - } + @Override + public boolean isControllerServiceEnabled(ControllerService service) { + return false; + } - @Override - public Set getControllerServiceIdentifiers(Class serviceType) - throws IllegalArgumentException { - return Collections.emptySet(); - } + @Override + public Set getControllerServiceIdentifiers(Class serviceType) + throws IllegalArgumentException { + return Collections.emptySet(); + } } diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/mock/MockProcessorInitializationContext.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/mock/MockProcessorInitializationContext.java index 48ffecb1ce..d77d7dde0f 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/mock/MockProcessorInitializationContext.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/mock/MockProcessorInitializationContext.java @@ -23,23 +23,23 @@ import org.apache.nifi.processor.ProcessorInitializationContext; /** * 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 ""; - } + @Override + public String getIdentifier() { + return ""; + } - @Override - public ProcessorLog getLogger() { - return null; - } + @Override + public ProcessorLog getLogger() { + return null; + } - @Override - public ControllerServiceLookup getControllerServiceLookup() { - return new MockControllerServiceLookup(); - } + @Override + public ControllerServiceLookup getControllerServiceLookup() { + return new MockControllerServiceLookup(); + } } diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/mock/MockReportingInitializationContext.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/mock/MockReportingInitializationContext.java index 9782077bc9..910ce5a5e2 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/mock/MockReportingInitializationContext.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/mock/MockReportingInitializationContext.java @@ -25,39 +25,39 @@ import org.apache.nifi.scheduling.SchedulingStrategy; /** * A Mock ReportingInitializationContext that can be used to initialize a * ReportingTask for the purposes of documentation generation. - * + * * @author Alligator * */ public class MockReportingInitializationContext implements ReportingInitializationContext { - @Override - public String getIdentifier() { - return ""; - } + @Override + public String getIdentifier() { + return ""; + } - @Override - public String getName() { - return ""; - } + @Override + public String getName() { + return ""; + } - @Override - public long getSchedulingPeriod(TimeUnit timeUnit) { - return 0; - } + @Override + public long getSchedulingPeriod(TimeUnit timeUnit) { + return 0; + } - @Override - public ControllerServiceLookup getControllerServiceLookup() { - return new MockControllerServiceLookup(); - } + @Override + public ControllerServiceLookup getControllerServiceLookup() { + return new MockControllerServiceLookup(); + } - @Override - public String getSchedulingPeriod() { - return ""; - } + @Override + public String getSchedulingPeriod() { + return ""; + } - @Override - public SchedulingStrategy getSchedulingStrategy() { - return SchedulingStrategy.TIMER_DRIVEN; - } + @Override + public SchedulingStrategy getSchedulingStrategy() { + return SchedulingStrategy.TIMER_DRIVEN; + } } diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/java/org/apache/nifi/documentation/example/NakedProcessor.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/java/org/apache/nifi/documentation/example/NakedProcessor.java index 6fce1e1386..ee7f315931 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/java/org/apache/nifi/documentation/example/NakedProcessor.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/test/java/org/apache/nifi/documentation/example/NakedProcessor.java @@ -1,3 +1,19 @@ +/* + * 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.example; import org.apache.nifi.processor.AbstractProcessor; @@ -9,7 +25,6 @@ public class NakedProcessor extends AbstractProcessor { @Override public void onTrigger(ProcessContext arg0, ProcessSession arg1) throws ProcessException { - // TODO Auto-generated method stub } diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/pom.xml b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/pom.xml index b0ed5c1969..c0b4ab9af0 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/pom.xml +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/pom.xml @@ -39,7 +39,7 @@ nifi-administration nifi-web nifi-resources - nifi-documentation + nifi-documentation diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/pom.xml b/nifi/nifi-nar-bundles/nifi-framework-bundle/pom.xml index 4c07d7562c..074821a7fb 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/pom.xml +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/pom.xml @@ -111,7 +111,7 @@ org.apache.nifi nifi-documentation - 0.0.2-incubating-SNAPSHOT + 0.1.0-incubating-SNAPSHOT diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/DistributeLoad.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/DistributeLoad.java index 8fb8ad485d..c78251c5b4 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/DistributeLoad.java +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/DistributeLoad.java @@ -151,7 +151,7 @@ public class DistributeLoad extends AbstractProcessor { } private static Relationship createRelationship(final int num) { - return new Relationship.Builder().name(String.valueOf(num)).build(); + return new Relationship.Builder().name(String.valueOf(num)).description("Where to route flowfiles for this relationship index").build(); } @Override diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/HandleHttpRequest.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/HandleHttpRequest.java index 575bf95027..4386100515 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/HandleHttpRequest.java +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/HandleHttpRequest.java @@ -79,9 +79,9 @@ public class HandleHttpRequest extends AbstractProcessor { private static final Pattern URL_QUERY_PARAM_DELIMITER = Pattern.compile("&"); // Allowable values for client auth - public static final AllowableValue CLIENT_NONE = new AllowableValue("No Authentication", "Processor will not authenticate clients. Anyone can communicate with this Processor anonymously"); - public static final AllowableValue CLIENT_WANT = new AllowableValue("Want Authentication", "Processor will try to verify the client but if unable to verify will allow the client to communicate anonymously"); - public static final AllowableValue CLIENT_NEED = new AllowableValue("Need Authentication", "Processor will reject communications from any client unless the client provides a certificate that is trusted by the TrustStore specified in the SSL Context Service"); + public static final AllowableValue CLIENT_NONE = new AllowableValue("No Authentication", "No Authentication", "Processor will not authenticate clients. Anyone can communicate with this Processor anonymously"); + public static final AllowableValue CLIENT_WANT = new AllowableValue("Want Authentication", "Want Authentication", "Processor will try to verify the client but if unable to verify will allow the client to communicate anonymously"); + public static final AllowableValue CLIENT_NEED = new AllowableValue("Need Authentication", "Need Authentication", "Processor will reject communications from any client unless the client provides a certificate that is trusted by the TrustStore specified in the SSL Context Service"); public static final PropertyDescriptor PORT = new PropertyDescriptor.Builder() @@ -208,6 +208,7 @@ public class HandleHttpRequest extends AbstractProcessor { descriptors.add(ALLOW_HEAD); descriptors.add(ALLOW_OPTIONS); descriptors.add(ADDITIONAL_METHODS); + descriptors.add(CLIENT_AUTH); return descriptors; } diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.DistributeLoad/additionalDetails.html b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.DistributeLoad/additionalDetails.html index 0c16e4fc7f..f593bed4b4 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.DistributeLoad/additionalDetails.html +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.DistributeLoad/additionalDetails.html @@ -22,39 +22,19 @@ -

      Description:

      -

      This processor distributes FlowFiles to downstream - processors based on a distribution strategy. The user may select the strategy "round robin", the - strategy "next available", or "load distribution service". If using the round robin - strategy, the default is to assign each destination (i.e., relationship) a weighting of 1 - (evenly distributed). However, the user may add optional properties to change this weighting. When - adding a property, the name must be a positive integer between 1 and the number of relationships (inclusive). - For example, if Number of Relationships has a value of 8 and a property is added with the name 5 - and the value 10, then relationship 5 (among the 8) will receive 10 - FlowFiles in each iteration instead of 1. All other relationships will receive 1 FlowFile in each iteration. -

      - Properties: -

      In the list below, the names of required properties appear - in bold. Any other properties (not in bold) are considered optional. - If a property has a default value, it is indicated. If a property - supports the use of the NiFi Expression Language (or simply, - "expression language"), that is also indicated.

      - -

      - Relationships: -

      -
        -
      • 1 -
          -
        • By default, there is one relationship, named 1. The number - of relationships is determined by the value of the <Number of - Relationships> property.
        • -
        -
      • -
      + Relationships +
        +
      • 1 +
          +
        • By default, there is one relationship, named 1. The number + of relationships is determined by the value of the <Number of + Relationships> property.
        • +
        +
      • +
      - + diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.EvaluateJsonPath/additionalDetails.html b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.EvaluateJsonPath/additionalDetails.html new file mode 100644 index 0000000000..c1d80aa43d --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.EvaluateJsonPath/additionalDetails.html @@ -0,0 +1,58 @@ + + + + + + EvaluateJsonPath + + + + + +

      + Note: The underlying JsonPath library loads the entirety of the streamed content into and performs + result evaluations in memory. Accordingly, it is important to consider the anticipated profile of content being + evaluated by this processor and the hardware supporting it especially when working against large JSON documents. +

      + +

      + Modifies Attributes: +

      + +

      + This processor adds user-defined attributes if the <Destination> property is set to + flowfile-attribute. +

      + + +

      + Properties +

      +
        +
      • + user-defined properties +
          +
        • The name of the attribute to put the JsonPath result into if + flowfile-attribute is used as the value for the Destination + property; if using flowfile-content as the value for the + Destination property, this value is ignored. +
        • +
        • Supports expression language: false
        • +
        +
      • +
      + + diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.EvaluateJsonPath/index.html b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.EvaluateJsonPath/index.html deleted file mode 100644 index 82f9614a37..0000000000 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.EvaluateJsonPath/index.html +++ /dev/null @@ -1,155 +0,0 @@ - - - - - - EvaluateJsonPath - - - - - -

      Description:

      - -

      - Evaluates one or more JsonPath expressions against the content of a FlowFile. The results of those expressions are - assigned to FlowFile Attributes or are written to the content of the FlowFile itself, depending on configuration of - the Processor. JsonPaths are entered by adding user-defined properties; the name of the property maps to the - Attribute Name into which the result will be placed (if the Destination is flowfile-attribute; otherwise, the - property name is ignored). The value of the property must be a valid JsonPath expression. If the JsonPath evaluates - to a JSON array or JSON object and the Return Type is set to 'scalar' the FlowFile will be unmodified and will be - routed to failure. A Return Type of JSON can return scalar values if the provided JsonPath evaluates to the - specified value and will be routed as a match. If Destination is 'flowfile-content' and the JsonPath does not - evaluate to a defined path, the FlowFile will be routed to 'unmatched' without having its contents modified. If - Destination is flowfile-attribute and the expression matches nothing, attributes will be created with empty - strings as the value, and the FlowFile will always be routed to 'matched.' -

      - -

      - Note: The underlying JsonPath library loads the entirety of the streamed content into and performs - result evaluations in memory. Accordingly, it is important to consider the anticipated profile of content being - evaluated by this processor and the hardware supporting it especially when working against large JSON documents. -

      - -

      - Properties: -

      - -

      - In the list below, the names of required properties appear in bold. - Any other properties (not in bold) are considered optional. If a - property has a default value, it is indicated. If a property - supports the use of the NiFi Expression Language (or simply, - "expression language"), that is also indicated. -

      - -

      - Modifies Attributes: -

      - -

      - This processor adds user-defined attributes if the <Destination> property is set to - flowfile-attribute. -

      - - -
        -
      • - Destination -
          -
        • Indicates whether the results of the JsonPath evaluation are - written to the FlowFile content or a FlowFile attribute; if using - attribute, the attribute's name must be specified in the value of - the Attribute Name property. -
        • -
        • - Valid values are: -
            -
          • flowfile-content
          • -
          • flowfile-attribute
          • -
          -
        • -
        • Default value: flowfile-content
        • -
        • Supports expression language: false
        • -
        -
      • -
      • - Return Type -
          -
        • Indicates the desired return type of the JsonPath expressions. - Selecting 'auto-detect' will set the return type to 'json' for a - Destination of 'flowfile-content', and 'scalar' for a Destination of - 'flowfile-attribute'.") -
        • -
        • - Valid values are: -
            -
          • auto-detect
          • -
          • json
          • -
          • scalar
          • -
          -
        • -
        • Default value: auto-detect
        • -
        • Supports expression language: false
        • -
        -
      • -
      • - user-defined properties -
          -
        • The name of the attribute to put the JsonPath result into if - flowfile-attribute is used as the value for the Destination - property; if using flowfile-content as the value for the - Destination property, this value is ignored. -
        • -
        • Supports expression language: false
        • -
        -
      • -
      - -

      - Relationships: -

      -
        -
      • - failure -
          -
        • If the JsonPath cannot be evaluated against the content of the - FlowFile, then the FlowFile follows this relationship. For - example, if the FlowFile does not contain valid JSON. -
        • -
        -
      • -
      • - matched -
          -
        • If the JsonPath is successfully evaluated and the FlowFile is - modified as a result, then the FlowFile follows this - relationship. -
        • -
        -
      • -
      • - unmatched -
          -
        • If the JsonPath does not match the content of the FlowFile, then - the FlowFile follows this relationship. -
        • -
        -
      • -
      -

      - - diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.EvaluateXQuery/additionalDetails.html b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.EvaluateXQuery/additionalDetails.html index 3d120a45b7..ae9e6ff3f7 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.EvaluateXQuery/additionalDetails.html +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.EvaluateXQuery/additionalDetails.html @@ -1,55 +1,55 @@ - - - -EvaluateXQuery - - + + + EvaluateXQuery + + - + - + -

      - Modifies Attributes: -

      +

      + Modifies Attributes: +

      -

      - This processor adds user-defined attributes if the <Destination> - property is set to - flowfile-attribute - . -

      +

      + This processor adds user-defined attributes if the <Destination> + property is set to + flowfile-attribute + . +

      -

      - Examples: -

      +

      + Examples: +

      -

      This processor produces one attribute or FlowFile per - XQueryResult. If only one attribute or FlowFile is desired, the - following examples demonstrate how this can be achieved using the - XQuery language. The examples below reference the following sample - XML:

      +

      This processor produces one attribute or FlowFile per + XQueryResult. If only one attribute or FlowFile is desired, the + following examples demonstrate how this can be achieved using the + XQuery language. The examples below reference the following sample + XML:

      -
      +        
       		
         <?xml version="1.0" encoding="UTF-8"?>
         <?xml-stylesheet type="text/xsl" href="foo.xsl"?>
      @@ -85,77 +85,77 @@
           </fruit>
         </ns:fruitbasket>
       
      -	
      +
      -

      -

        -
      • XQuery to return all "fruit" nodes individually (7 Results): -
          -
        • //fruit
        • -
        -
      • -
      • XQuery to return only the first "fruit" node (1 Result): -
          -
        • //fruit[1]
        • -
        -
      • -
      • XQuery to return only the last "fruit" node (1 Result): -
          -
        • //fruit[count(//fruit)]
        • -
        -
      • -
      • XQuery to return all "fruit" nodes, wrapped in a "basket" tag - (1 Result): -
          -
        • <basket>{//fruit}</basket>
        • -
        -
      • -
      • XQuery to return all "fruit" names individually (7 Results): -
          -
        • //fruit/text()
        • -
        -
      • -
      • XQuery to return only the first "fruit" name (1 Result): -
          -
        • //fruit[1]/text()
        • -
        -
      • -
      • XQuery to return only the last "fruit" name (1 Result): -
          -
        • //fruit[count(//fruit)]/text()
        • -
        -
      • -
      • XQuery to return all "fruit" names as a comma separated list - (1 Result): -
          -
        • string-join((for $x in //fruit return $x/name/text()), ', - ')
        • -
        -
      • -
      • XQuery to return all "fruit" colors and names as a comma - separated list (1 Result): -
          -
        • string-join((for $y in (for $x in //fruit return - string-join(($x/color/text() , $x/name/text()), ' ')) return $y), - ', ')
        • -
        -
      • -
      • XQuery to return all "fruit" colors and names as a comma - separated list (1 Result): -
          -
        • string-join((for $y in (for $x in //fruit return - string-join(($x/color/text() , $x/name/text()), ' ')) return $y), - ', ')
        • -
        -
      • -
      • XQuery to return all "fruit" colors and names as a new line - separated list (1 Result): -
          -
        • string-join((for $y in (for $x in //fruit return - string-join(($x/color/text() , $x/name/text()), ' ')) return $y), - '\n')
        • -
        -
      • -
      - +

      +

        +
      • XQuery to return all "fruit" nodes individually (7 Results): +
          +
        • //fruit
        • +
        +
      • +
      • XQuery to return only the first "fruit" node (1 Result): +
          +
        • //fruit[1]
        • +
        +
      • +
      • XQuery to return only the last "fruit" node (1 Result): +
          +
        • //fruit[count(//fruit)]
        • +
        +
      • +
      • XQuery to return all "fruit" nodes, wrapped in a "basket" tag + (1 Result): +
          +
        • <basket>{//fruit}</basket>
        • +
        +
      • +
      • XQuery to return all "fruit" names individually (7 Results): +
          +
        • //fruit/text()
        • +
        +
      • +
      • XQuery to return only the first "fruit" name (1 Result): +
          +
        • //fruit[1]/text()
        • +
        +
      • +
      • XQuery to return only the last "fruit" name (1 Result): +
          +
        • //fruit[count(//fruit)]/text()
        • +
        +
      • +
      • XQuery to return all "fruit" names as a comma separated list + (1 Result): +
          +
        • string-join((for $x in //fruit return $x/name/text()), ', + ')
        • +
        +
      • +
      • XQuery to return all "fruit" colors and names as a comma + separated list (1 Result): +
          +
        • string-join((for $y in (for $x in //fruit return + string-join(($x/color/text() , $x/name/text()), ' ')) return $y), + ', ')
        • +
        +
      • +
      • XQuery to return all "fruit" colors and names as a comma + separated list (1 Result): +
          +
        • string-join((for $y in (for $x in //fruit return + string-join(($x/color/text() , $x/name/text()), ' ')) return $y), + ', ')
        • +
        +
      • +
      • XQuery to return all "fruit" colors and names as a new line + separated list (1 Result): +
          +
        • string-join((for $y in (for $x in //fruit return + string-join(($x/color/text() , $x/name/text()), ' ')) return $y), + '\n')
        • +
        +
      • +
      + diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.ExecuteProcess/index.html b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.ExecuteProcess/index.html deleted file mode 100644 index 3526cd1526..0000000000 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.ExecuteProcess/index.html +++ /dev/null @@ -1,100 +0,0 @@ - - - - - - ExecuteProcess - - - - - - -

      Description:

      -

      - Runs an operating system command specified by the user and writes the output of that command to a FlowFile. If the command is expected - to be long-running, the Processor can output the partial data on a specified interval. When this option is used, the output is expected to be in textual - format, as it typically does not make sense to split binary data on arbitrary time-based intervals. -

      - -

      - Properties: -

      -

      In the list below, the names of required properties appear - in bold. Any other properties (not in bold) are considered optional. - If a property has a default value, it is indicated. If a property - supports the use of the NiFi Expression Language (or simply, - "expression language"), that is also indicated.

      -
        -
      • Command -
          -
        • Specifies the command to be executed; if just the name of an executable is provided, it must be in the user's environment PATH.
        • -
        • Default value: none
        • -
        • Supports expression language: false
        • -
        -
      • -
      • Command Arguments -
          -
        • The arguments to supply to the executable delimited by white space. White space can be escaped by enclosing it in double-quotes.
        • -
        • Default value: none
        • -
        • Supports expression language: false
        • -
        -
      • -
      • Working Directory -
          -
        • The directory to use as the current working directory when executing the command
        • -
        • Default value: none (which means whatever NiFi's root installation directory is)
        • -
        • Supports expression language: false
        • -
        -
      • -
      • Batch Duration -
          -
        • - If the process is expected to be long-running and produce textual output, a batch duration can be specified so - that the output will be captured for this amount of time and a FlowFile will then be sent out with the results - and a new FlowFile will be started, rather than waiting for the process to finish before sending out the results. - If no value is provided, the process will run to completion and the entire output of the process will be written - to a single FlowFile. -
        • -
        • Default value: none
        • -
        • Supports expression language: false
        • -
        -
      • -
      • Redirect Error Stream -
          -
        • - If true will redirect any error stream output of the process to the output stream. - This is particularly helpful for processes which write extensively to the error stream or for troubleshooting. -
        • -
        • Default value: false
        • -
        • Allowed Values: true, false
        • -
        • Supports expression language: false
        • -
        -
      • -
      - -

      - Relationships: -

      -
        -
      • success -
          -
        • All FlowFiles that are created are routed to this relationship.
        • -
        -
      • -
      - - diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.HandleHttpRequest/additionalDetails.html b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.HandleHttpRequest/additionalDetails.html new file mode 100644 index 0000000000..17378fb3cc --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.HandleHttpRequest/additionalDetails.html @@ -0,0 +1,127 @@ + + + + + + HandleHttpRequest + + + + +

      Usage Description

      +

      + The pairing of this Processor with a HandleHttpResponse Processor + provides the ability to use NiFi to visually construct a web server that can carry out any functionality that is available + through the existing Processors. For example, one could construct a Web-based front end to an SFTP Server by constructing a + flow such as: +

      + +

      + HandleHttpRequest -> + PutSFTP -> + HandleHttpResponse +

      + +

      + The HandleHttpRequest Processor provides several Properties to configure which methods are supported, the paths that are + supported, and SSL configuration. The FlowFiles that are generated by this Processor have the following attributes added to + them, providing powerful routing capabilities and traceability of all data: +

      + +
      Attribute Name
      + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
      Attribute NameAttribute Description
      http.context.identifierAn identifier that allows the HandleHttpRequest and HandleHttpResponse to coordinate which FlowFile belongs + to which HTTP Request/Response.
      mime.typeThe MIME Type of the data, according to the HTTP Header "Content-Type"
      http.servlet.pathThe part of the request URL that is considered the Servlet Path
      http.context.pathThe part of the request URL that is considered to be the Context Path
      http.methodThe HTTP Method that was used for the request, such as GET or POST
      http.query.stringThe query string portion of hte Request URL
      http.remote.hostThe hostname of the requestor
      http.remote.addrThe hostname:port combination of the requestor
      http.remote.userThe username of the requestor
      http.request.uriThe full Request URL
      http.auth.typeThe type of HTTP Authorization used
      http.principal.nameThe name of the authenticated user making the request
      http.subject.dnThe Distinguished Name of the requestor. This value will not be populated unless the Processor is + configured to use an SSLContext Service
      http.issuer.dnThe Distinguished Name of the entity that issued the Subject's certificate. This value will not be + populated unless the Processor is configured to use an SSLContext Service
      http.headers.XXXEach of the HTTP Headers that is received in the request will be added as an attribute, prefixed + with "http.headers." For example, if the request contains an HTTP Header named "x-my-header", + then the value will be added to an attribute named "http.headers.x-my-header"
      +

      + See Also:
      + HandleHttpResponse
      + StandardHttpContextMap
      + StandardSSLContextService
      +

      + + + diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.HandleHttpRequest/index.html b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.HandleHttpRequest/index.html deleted file mode 100644 index d3da666686..0000000000 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.HandleHttpRequest/index.html +++ /dev/null @@ -1,255 +0,0 @@ - - - - - - HandleHttpRequest - - - - - -

      Description:

      -

      - This processor starts an HTTP server and creates a FlowFile for each HTTP Request that it receives. The Processor leaves - the HTTP Connection open and is intended to be used in conjunction with a - HandleHttpResponse Processor. -

      - -

      - The pairing of this Processor with a HandleHttpResponse Processor - provides the ability to use NiFi to visually construct a web server that can carry out any functionality that is available - through the existing Processors. For example, one could construct a Web-based front end to an SFTP Server by constructing a - flow such as: -

      - -

      - HandleHttpRequest -> - PutSFTP -> - HandleHttpResponse -

      - -

      - The HandleHttpRequest Processor provides several Properties to configure which methods are supported, the paths that are - supported, and SSL configuration. The FlowFiles that are generated by this Processor have the following attributes added to - them, providing powerful routing capabilities and traceability of all data: -

      - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
      Attribute NameAttribute Description
      http.context.identifierAn identifier that allows the HandleHttpRequest and HandleHttpResponse to coordinate which FlowFile belongs - to which HTTP Request/Response.
      mime.typeThe MIME Type of the data, according to the HTTP Header "Content-Type"
      http.servlet.pathThe part of the request URL that is considered the Servlet Path
      http.context.pathThe part of the request URL that is considered to be the Context Path
      http.methodThe HTTP Method that was used for the request, such as GET or POST
      http.query.stringThe query string portion of hte Request URL
      http.remote.hostThe hostname of the requestor
      http.remote.addrThe hostname:port combination of the requestor
      http.remote.userThe username of the requestor
      http.request.uriThe full Request URL
      http.auth.typeThe type of HTTP Authorization used
      http.principal.nameThe name of the authenticated user making the request
      http.subject.dnThe Distinguished Name of the requestor. This value will not be populated unless the Processor is - configured to use an SSLContext Service
      http.issuer.dnThe Distinguished Name of the entity that issued the Subject's certificate. This value will not be - populated unless the Processor is configured to use an SSLContext Service
      http.headers.XXXEach of the HTTP Headers that is received in the request will be added as an attribute, prefixed - with "http.headers." For example, if the request contains an HTTP Header named "x-my-header", - then the value will be added to an attribute named "http.headers.x-my-header"
      - - -

      - Properties: -

      -

      - In the list below, the names of required properties appear in bold. Any other properties (not in bold) are considered optional. If a property has a default value, it is indicated. If a property supports the use of the NiFi Expression Language (or simply, "expression language"), that is also indicated. -

      -
        -
      • Listening Port -
          -
        • The port to listen on for incoming HTTP Requests
        • -
        • Default value: no default
        • -
        • Supports expression language: false
        • -
        -
      • -
      • Hostname -
          -
        • The Hostname to bind to. If not specified, will bind to all hosts
        • -
        • Default value: no default
        • -
        • Supports expression language: false
        • -
        -
      • -
      • HTTP Context Map -
          -
        • The HTTP Context Map Controller Service to use for caching the HTTP Request Information
        • -
        • Default value: no default
        • -
        • Supports expression language: false
        • -
        -
      • -
      • SSL Context Service -
          -
        • The Controller Service to use for obtaining an SSL Context. The SSL Context controller service is a mechanism for providing all the security properties that allow for secure communications between NiFi extensions and other systems. See the User Guide or the Controller Services documentation via the "help" link in the upper-right corner of the GUI for more information about the StandardSSLContextService. The value for this property is the identifier name that is configured in the StandardSSLContextService.
        • -
        • Default value: no default
        • -
        • Supports expression language: false
        • -
        -
      • - -
      • Allowed Paths -
          -
        • A Regular Expression that specifies the valid HTTP Paths that are allowed in the incoming URL Requests. If this value is specified and the path of the HTTP Requests does not match this Regular Expression, the Processor will respond with a 404: NotFound
        • -
        • Default value: no default
        • -
        • Supports expression language: false
        • -
        -
      • -
      • Allow GET -
          -
        • Specifies whether or not to allow HTTP GET Method
        • -
        • Default value: true
        • -
        • Supports expression language: false
        • -
        -
      • -
      • Allow POST -
          -
        • Specifies whether or not to allow HTTP POST Method
        • -
        • Default value: true
        • -
        • Supports expression language: false
        • -
        -
      • -
      • Allow PUT -
          -
        • Specifies whether or not to allow HTTP PUT Method
        • -
        • Default value: true
        • -
        • Supports expression language: false
        • -
        -
      • -
      • Allow DELETE -
          -
        • Specifies whether or not to allow HTTP DELETE Method
        • -
        • Default value: true
        • -
        • Supports expression language: false
        • -
        -
      • -
      • Allow HEAD -
          -
        • Specifies whether or not to allow HTTP HEAD Method
        • -
        • Default value: false
        • -
        • Supports expression language: false
        • -
        -
      • -
      • Allow OPTIONS -
          -
        • Specifies whether or not to allow HTTP OPTIONS Method
        • -
        • Default value: false
        • -
        • Supports expression language: false
        • -
        -
      • -
      • Additional HTTP Methods -
          -
        • A comma-separated list of non-standard HTTP Methods that should be allowed
        • -
        • Default value: no default
        • -
        • Supports expression language: false
        • -
        -
      • -
      • Client Authentication -
          -
        • Specifies whether or not the Processor should authenticate clients. This value is ignored if the <SSL Context Service> Property is not specified or the SSL Context provided uses only a KeyStore and not a TrustStore.
        • -
        • - The following values are allowed: -
            -
          • No Authentication - Processor will not authenticate clients. Anyone can communicate with this Processor anonymously
          • -
          • Want Authentication - Processor will try to verify the client but if unable to verify will allow the client to communicate anonymously
          • -
          • Need Authentication - Processor will reject communications from any client unless the client provides a certificate that is trusted by the TrustStore specified in the SSL Context Service
          • -
          -
        • -
        • Default value: No Authentication
        • -
        • Supports expression language: false
        • -
        -
      • -
      - -

      - Relationships: -

      -
        -
      • success -
          -
        • All FlowFiles that are created are routed to this relationship.
        • -
        -
      • -
      - - -

      - See Also:
      - HandleHttpResponse
      - StandardHttpContextMap
      - StandardSSLContextService
      -

      - - - diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.HandleHttpResponse/additionalDetails.html b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.HandleHttpResponse/additionalDetails.html new file mode 100644 index 0000000000..5d8993e6c7 --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.HandleHttpResponse/additionalDetails.html @@ -0,0 +1,57 @@ + + + + + + HandleHttpResponse + + + + + +

      Usage Description:

      +

      + The pairing of this Processor with a HandleHttpRequest Processor + provides the ability to use NiFi to visually construct a web server that can carry out any functionality that is available + through the existing Processors. For example, one could construct a Web-based front end to an SFTP Server by constructing a + flow such as: +

      + +

      + HandleHttpRequest -> + PutSFTP -> + HandleHttpResponse +

      + +

      + This Processor must be configured with the same <HTTP Context Map> service as the corresponding HandleHttpRequest Processor. + Otherwise, all FlowFiles will be routed to the 'failure' relationship. +

      + +

      + All FlowFiles must have an attribute named http.context.identifier. The value of this attribute is used to lookup + the HTTP Response so that the proper message can be sent back to the requestor. If this attribute is missing, the FlowFile + will be routed to 'failure.' +

      +

      + See Also:
      + HandleHttpRequest
      + StandardHttpContextMap
      + StandardSSLContextService
      +

      + + + diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.HandleHttpResponse/index.html b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.HandleHttpResponse/index.html deleted file mode 100644 index 70d76a6684..0000000000 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.HandleHttpResponse/index.html +++ /dev/null @@ -1,112 +0,0 @@ - - - - - - HandleHttpResponse - - - - - -

      Description:

      -

      - This processor responds to an HTTP request that was received by the - HandleHttpRequest Processor. -

      - -

      - The pairing of this Processor with a HandleHttpRequest Processor - provides the ability to use NiFi to visually construct a web server that can carry out any functionality that is available - through the existing Processors. For example, one could construct a Web-based front end to an SFTP Server by constructing a - flow such as: -

      - -

      - HandleHttpRequest -> - PutSFTP -> - HandleHttpResponse -

      - -

      - This Processor must be configured with the same <HTTP Context Map> service as the corresponding HandleHttpRequest Processor. - Otherwise, all FlowFiles will be routed to the 'failure' relationship. -

      - -

      - All FlowFiles must have an attribute named http.context.identifier. The value of this attribute is used to lookup - the HTTP Response so that the proper message can be sent back to the requestor. If this attribute is missing, the FlowFile - will be routed to 'failure.' -

      - - -

      - Properties: -

      -

      - In the list below, the names of required properties appear in bold. Any other properties (not in bold) are considered optional. If a property has a default value, it is indicated. If a property supports the use of the NiFi Expression Language (or simply, "expression language"), that is also indicated. -

      -
        -
      • HTTP Status Code -
          -
        • The HTTP Status Code to use when responding to the HTTP Request. See Section 10 of RFC 2616 for more information.
        • -
        • Default value: no default
        • -
        • Supports expression language: true
        • -
        -
      • -
      • HTTP Context Map -
          -
        • The HTTP Context Map Controller Service to use for caching the HTTP Request Information
        • -
        • Default value: no default
        • -
        • Supports expression language: false
        • -
        -
      • -
      - -

      - Relationships: -

      -
        -
      • success -
          -
        • If a message is successfully sent back to the requestor, the FlowFile is routed to this relationship.
        • -
        -
      • - -
      • failure -
          -
        • - A FlowFile will be routed to failure under the following conditions: -
            -
          • The FlowFile does not have an http.context.identifier attribute
          • -
          • The http.context.identifier attribute has a value that cannot be found in the HTTP Context Map
          • -
          • The HTTP Status Code is not a number
          • -
          • There was a communications failure when attempting to write the response to the requestor
          • -
          -
        • -
        -
      • -
      - -

      - See Also:
      - HandleHttpRequest
      - StandardHttpContextMap
      - StandardSSLContextService
      -

      - - - diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.IdentifyMimeType/additionalDetails.html b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.IdentifyMimeType/additionalDetails.html index 22ebd1db03..f993fee963 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.IdentifyMimeType/additionalDetails.html +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.IdentifyMimeType/additionalDetails.html @@ -22,77 +22,23 @@ - -

      Description:

      - This processor attempts to identify the MIME type used for a - FlowFile. If the MIME type can be identified, an attribute with the - name - mime.type - is added to the FlowFile, and its value is the detected MIME type. - Some MIME - types require the processor to read a significant amount of data; for - these MIME types, their identification is optional. (See the - properties Identify ZIP and Identify TAR.) The algorithm may have to - read the entire contents of a file for each type of identification. If - the MIME Type cannot be determined, its mime.type attribute will be - set to - application/octet-stream - . + Modifies Attributes:

      - -

      The following MIME Types are detected: -

      -
        -
      • application/gzip
      • -
      • application/bzip2
      • -
      • application/flowfile-v3
      • -
      • application/flowfile-v1 (requires Identify TAR be set to true)
      • -
      • application/xml
      • -
      • video/mp4
      • -
      • video/x-m4v
      • -
      • video/mp4a-latm
      • -
      • video/quicktime
      • -
      • video/mpeg
      • -
      • audio/wav
      • -
      • audio/mp3
      • -
      • image/bmp
      • -
      • image/png
      • -
      • image/jpg
      • -
      • image/gif
      • -
      • image/tif
      • -
      • application/vnd.ms-works
      • -
      • application/msexcel
      • -
      • application/mspowerpoint
      • -
      • application/msaccess
      • -
      • application/x-ms-wmv
      • -
      • application/pdf
      • -
      • application/x-rpm
      • -
      • application/tar
      • -
      • application/x-7z-compressed
      • -
      • application/java-archive
      • -
      • application/zip
      • -
      • application/x-lzh
      • -
      - - -

      - Modifies Attributes: -

      - - - - - - - - - - - - - -
      Attribute NameDescription
      mime.typeThis Processor sets the FlowFile's mime.type attribute to the detected MIME Type. If unable to detect - the MIME Type, the attribute's value will be set to application/octet-stream.
      - + + + + + + + + + + + + + +
      Attribute NameDescription
      mime.typeThis Processor sets the FlowFile's mime.type attribute to the detected MIME Type. If unable to detect + the MIME Type, the attribute's value will be set to application/octet-stream.
      + diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.InvokeHTTP/additionalDetails.html b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.InvokeHTTP/additionalDetails.html index 1c33360d26..9a18355b11 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.InvokeHTTP/additionalDetails.html +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.InvokeHTTP/additionalDetails.html @@ -21,7 +21,7 @@ - +

      Adds Attributes:

      diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.MonitorActivity/additionalDetails.html b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.MonitorActivity/additionalDetails.html index 0a3b66537e..d55fcec4d8 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.MonitorActivity/additionalDetails.html +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.MonitorActivity/additionalDetails.html @@ -23,7 +23,7 @@ - +

      Modifies Attributes:

      diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.PostHTTP/additionalDetails.html b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.PostHTTP/additionalDetails.html index f8b15f91c5..d193e93be9 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.PostHTTP/additionalDetails.html +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.PostHTTP/additionalDetails.html @@ -25,7 +25,7 @@ - +

      Uses Attributes:

      diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.PutFTP/additionalDetails.html b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.PutFTP/additionalDetails.html index fbece63817..d8afa01da0 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.PutFTP/additionalDetails.html +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.PutFTP/additionalDetails.html @@ -22,7 +22,7 @@ - +

      Optional User-Defined Properties:

      @@ -37,30 +37,12 @@ essentially the same as sending quote commands to an FTP server from the command line.
    • NOTE: While this is the same as sending a quote command, it is very important that - you leave off the “quote” part of the command from the text value you enter in. For example, if you want - to mimic the command line ftp command quote stat you would give the text value of stat. You have the - option of sending a number of commands to the FTP server before the file is sent. If you have only one - command, then the name of the optional parameter will be pre.cmd.1. If you have a second command, then - the next optional parameter you give is pre.cmd.2, and so on. There is no limit to the number of pre - commands you can send. The commands will be sent in the order of their number, so pre.cmd.1 will be sent - first, pre.cmd.2 will be sent next, etc. This is true regardless of what order they are added to the - processor properties. The property names determine the order.
    • -
    • NOTE: If the optional property name does not follow the pattern pre.cmd.integer - - then the command will not be sent. If a command is given which is not recognized by the server, then - that will be logged.
    • -
    • Supports expression language: true. FlowFile attributes can be used in commands using the expression language
    • -
    - -
  • post.cmd.# -
      -
    • Optional properties of this type can be added by the user and should be used for VERY - RARE cases only. You will know when you need to use it when the recipient specifies that you must - send FTP commands after the transfer. Otherwise, do not use it. These are the same as the pre commands. + you leave off the . (See pre commands above) except that these commands are sent after the file transfer.
    • Supports expression language: true. FlowFile attributes can be used in commands using the expression language
  • - + diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.PutSFTP/additionalDetails.html b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.PutSFTP/additionalDetails.html index 656e150ebc..93876b62e7 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.PutSFTP/additionalDetails.html +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.PutSFTP/additionalDetails.html @@ -24,11 +24,11 @@

    See Also: -

    +

    - + diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.RouteOnAttribute/additionalDetails.html b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.RouteOnAttribute/additionalDetails.html index 9bd3c383f9..92f9f46b30 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.RouteOnAttribute/additionalDetails.html +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.RouteOnAttribute/additionalDetails.html @@ -22,7 +22,7 @@ -

    Description:

    +

    Usage Example

    This processor routes FlowFiles based on their attributes using the NiFi Expression Language. Users add properties with diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.ScanContent/additionalDetails.html b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.ScanContent/additionalDetails.html index f8560f2d0f..aff65491d3 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.ScanContent/additionalDetails.html +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.ScanContent/additionalDetails.html @@ -23,7 +23,7 @@ - +

    Modifies Attributes:

    diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.SegmentContent/additionalDetails.html b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.SegmentContent/additionalDetails.html index 64dd7baec3..957913f0e5 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.SegmentContent/additionalDetails.html +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.SegmentContent/additionalDetails.html @@ -23,7 +23,7 @@ - +

    Adds or Modifies Attributes:

    diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.SplitContent/additionalDetails.html b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.SplitContent/additionalDetails.html index 64d8d7dce3..683b0fa5a1 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.SplitContent/additionalDetails.html +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.SplitContent/additionalDetails.html @@ -23,7 +23,7 @@ - +

    Modifies Attributes:

    diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.SplitJson/index.html b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.SplitJson/index.html deleted file mode 100644 index e3a4b14e66..0000000000 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.SplitJson/index.html +++ /dev/null @@ -1,85 +0,0 @@ - - - - - - SplitJson - - - - - - -

    Description:

    - -

    - This processor splits a JSON File into multiple, separate FlowFiles for an array element specified by a JsonPath - expression. - Each generated FlowFile is comprised of an element of the specified array and transferred to relationship 'split,' - with the original file transferred to the 'original' relationship. If the specified JsonPath is not found or - does not evaluate to an array element, the original file is routed to 'failure' and no files are generated. -

    - -

    - Note: The underlying JsonPath library loads the entirety of the streamed content into and performs - result evaluations in memory. Accordingly, it is important to consider the anticipated profile of content being - evaluated by this processor and the hardware supporting it especially when working against large JSON documents. -

    - -Properties: -

    - -

    - In the list below, the names of required properties appear in bold. Any other properties (not in bold) are - considered optional. If a property has a default value, it is indicated. If a property supports the use of the NiFi - Expression Language (or simply, "expression language"), that is also indicated. -

    -
      -
    • JsonPath Expression -
        -
      • A JsonPath expression that indicates the array element to split into JSON/scalar fragments.
      • -
      • Supports expression language: false
      • -
      -
    • -
    - -

    - Relationships: -

    -
      -
    • failure -
        -
      • If a FlowFile fails processing for any reason (for example, the FlowFile is not valid JSON or the - specified path does not exist) and does not go down the original relationship. -
      • -
      -
    • -
    • original -
        -
      • If FlowFiles are successfully split, a copy of the original FlowFile follows this relationship.
      • -
      -
    • -
    • split -
        -
      • If FlowFiles are successfully split into one or more files, those split files follow this - relationship. -
      • -
      -
    • -
    - - - From c118ead0bbaa8cd477cde99262d56529a176d41f Mon Sep 17 00:00:00 2001 From: joewitt Date: Wed, 18 Mar 2015 00:01:19 -0400 Subject: [PATCH 26/26] This closes #1 Feedback on the PR has been provided in NIFI-190