From d7cf25be9a157f6934593a325f906a0fc85e4007 Mon Sep 17 00:00:00 2001 From: danbress Date: Sat, 31 Jan 2015 09:02:23 -0500 Subject: [PATCH 001/116] 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 002/116] 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 003/116] 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 004/116] 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 005/116] 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 006/116] 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 007/116] 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 008/116] 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 009/116] 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 010/116] 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 011/116] 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 012/116] 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 013/116] 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 014/116] 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 015/116] 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 016/116] 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 017/116] 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 018/116] 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 019/116] 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 020/116] 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 021/116] 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 022/116] 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 023/116] 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 024/116] 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 025/116] 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 026/116] This closes #1 Feedback on the PR has been provided in NIFI-190 From b560a88cbde48e669cee89e2c6cb18fcae736c72 Mon Sep 17 00:00:00 2001 From: Mark Payne Date: Wed, 18 Mar 2015 10:10:21 -0400 Subject: [PATCH 027/116] NIFI-432: Reduced amount of time needed for rollovers in unit tests in order to speed up tests --- .../PersistentProvenanceRepository.java | 10 +++-- .../TestPersistentProvenanceRepository.java | 40 +++++++++---------- 2 files changed, 26 insertions(+), 24 deletions(-) diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/PersistentProvenanceRepository.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/PersistentProvenanceRepository.java index f46b5fc16f..0502cc7ef4 100644 --- a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/PersistentProvenanceRepository.java +++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/PersistentProvenanceRepository.java @@ -130,6 +130,7 @@ public class PersistentProvenanceRepository implements ProvenanceEventRepository private final RepositoryConfiguration configuration; private final IndexConfiguration indexConfig; private final boolean alwaysSync; + private final int rolloverCheckMillis; private final ScheduledExecutorService scheduledExecService; private final ExecutorService rolloverExecutor; @@ -156,10 +157,10 @@ public class PersistentProvenanceRepository implements ProvenanceEventRepository private EventReporter eventReporter; public PersistentProvenanceRepository() throws IOException { - this(createRepositoryConfiguration()); + this(createRepositoryConfiguration(), 10000); } - public PersistentProvenanceRepository(final RepositoryConfiguration configuration) throws IOException { + public PersistentProvenanceRepository(final RepositoryConfiguration configuration, final int rolloverCheckMillis) throws IOException { if (configuration.getStorageDirectories().isEmpty()) { throw new IllegalArgumentException("Must specify at least one storage directory"); } @@ -181,7 +182,8 @@ public class PersistentProvenanceRepository implements ProvenanceEventRepository this.maxPartitionBytes = configuration.getMaxEventFileCapacity(); this.indexConfig = new IndexConfiguration(configuration); this.alwaysSync = configuration.isAlwaysSync(); - + this.rolloverCheckMillis = rolloverCheckMillis; + final List fields = configuration.getSearchableFields(); if (fields != null && !fields.isEmpty()) { indexingAction = new IndexingAction(this, indexConfig); @@ -244,7 +246,7 @@ public class PersistentProvenanceRepository implements ProvenanceEventRepository } } } - }, 10L, 10L, TimeUnit.SECONDS); + }, rolloverCheckMillis, rolloverCheckMillis, TimeUnit.MILLISECONDS); scheduledExecService.scheduleWithFixedDelay(new RemoveExpiredQueryResults(), 30L, 3L, TimeUnit.SECONDS); scheduledExecService.scheduleWithFixedDelay(new Runnable() { diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/test/java/org/apache/nifi/provenance/TestPersistentProvenanceRepository.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/test/java/org/apache/nifi/provenance/TestPersistentProvenanceRepository.java index 730587296e..75e3514330 100644 --- a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/test/java/org/apache/nifi/provenance/TestPersistentProvenanceRepository.java +++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/test/java/org/apache/nifi/provenance/TestPersistentProvenanceRepository.java @@ -170,7 +170,7 @@ public class TestPersistentProvenanceRepository { config.setCompressOnRollover(false); config.setJournalCount(10); config.setQueryThreadPoolSize(10); - repo = new PersistentProvenanceRepository(config); + repo = new PersistentProvenanceRepository(config, 75); repo.initialize(getEventReporter()); final Map attributes = new HashMap<>(); @@ -218,7 +218,7 @@ public class TestPersistentProvenanceRepository { System.out.println("Closing and re-initializing"); repo.close(); - repo = new PersistentProvenanceRepository(config); + repo = new PersistentProvenanceRepository(config, 75); repo.initialize(getEventReporter()); System.out.println("Re-initialized"); @@ -238,7 +238,7 @@ public class TestPersistentProvenanceRepository { final RepositoryConfiguration config = createConfiguration(); config.setMaxEventFileCapacity(1L); config.setMaxEventFileLife(1, TimeUnit.SECONDS); - repo = new PersistentProvenanceRepository(config); + repo = new PersistentProvenanceRepository(config, 75); repo.initialize(getEventReporter()); final Map attributes = new HashMap<>(); @@ -262,7 +262,7 @@ public class TestPersistentProvenanceRepository { repo.close(); Thread.sleep(500L); // Give the repo time to shutdown (i.e., close all file handles, etc.) - repo = new PersistentProvenanceRepository(config); + repo = new PersistentProvenanceRepository(config, 75); repo.initialize(getEventReporter()); final List recoveredRecords = repo.getEvents(0L, 12); @@ -285,7 +285,7 @@ public class TestPersistentProvenanceRepository { config.setMaxEventFileCapacity(1024L * 1024L); config.setMaxEventFileLife(2, TimeUnit.SECONDS); config.setSearchableFields(searchableFields); - repo = new PersistentProvenanceRepository(config); + repo = new PersistentProvenanceRepository(config, 75); repo.initialize(getEventReporter()); final Map attributes = new HashMap<>(); @@ -339,7 +339,7 @@ public class TestPersistentProvenanceRepository { final RepositoryConfiguration config = createConfiguration(); config.setMaxEventFileLife(500, TimeUnit.MILLISECONDS); config.setSearchableFields(new ArrayList<>(SearchableFields.getStandardFields())); - repo = new PersistentProvenanceRepository(config); + repo = new PersistentProvenanceRepository(config, 75); repo.initialize(getEventReporter()); final String uuid = "00000000-0000-0000-0000-000000000000"; @@ -383,7 +383,7 @@ public class TestPersistentProvenanceRepository { final RepositoryConfiguration config = createConfiguration(); config.setMaxEventFileLife(500, TimeUnit.MILLISECONDS); config.setCompressOnRollover(true); - repo = new PersistentProvenanceRepository(config); + repo = new PersistentProvenanceRepository(config, 75); repo.initialize(getEventReporter()); final String uuid = "00000000-0000-0000-0000-000000000000"; @@ -421,7 +421,7 @@ public class TestPersistentProvenanceRepository { config.setMaxEventFileCapacity(1024L * 1024L); config.setSearchableFields(new ArrayList<>(SearchableFields.getStandardFields())); - repo = new PersistentProvenanceRepository(config); + repo = new PersistentProvenanceRepository(config, 75); repo.initialize(getEventReporter()); final String uuid = "10000000-0000-0000-0000-000000000000"; @@ -480,7 +480,7 @@ public class TestPersistentProvenanceRepository { config.setMaxEventFileCapacity(1024L * 1024L); config.setSearchableFields(new ArrayList<>(SearchableFields.getStandardFields())); - repo = new PersistentProvenanceRepository(config); + repo = new PersistentProvenanceRepository(config, 75); repo.initialize(getEventReporter()); final String uuid = "00000000-0000-0000-0000-000000000000"; @@ -543,7 +543,7 @@ public class TestPersistentProvenanceRepository { config.setMaxEventFileCapacity(1024L * 1024L); config.setSearchableFields(new ArrayList<>(SearchableFields.getStandardFields())); - repo = new PersistentProvenanceRepository(config); + repo = new PersistentProvenanceRepository(config, 75); repo.initialize(getEventReporter()); final String uuid = "00000000-0000-0000-0000-000000000000"; @@ -625,7 +625,7 @@ public class TestPersistentProvenanceRepository { config.setMaxEventFileCapacity(1024L * 1024L); config.setSearchableFields(new ArrayList<>(SearchableFields.getStandardFields())); - repo = new PersistentProvenanceRepository(config); + repo = new PersistentProvenanceRepository(config, 75); repo.initialize(getEventReporter()); final String uuid = "00000000-0000-0000-0000-000000000000"; @@ -681,7 +681,7 @@ public class TestPersistentProvenanceRepository { config.setMaxEventFileCapacity(1024L * 1024L); config.setSearchableFields(new ArrayList<>(SearchableFields.getStandardFields())); - repo = new PersistentProvenanceRepository(config); + repo = new PersistentProvenanceRepository(config, 75); repo.initialize(getEventReporter()); final String uuid = "00000000-0000-0000-0000-000000000001"; @@ -736,7 +736,7 @@ public class TestPersistentProvenanceRepository { config.setMaxEventFileCapacity(1024L * 1024L); config.setSearchableFields(new ArrayList<>(SearchableFields.getStandardFields())); - repo = new PersistentProvenanceRepository(config); + repo = new PersistentProvenanceRepository(config, 75); repo.initialize(getEventReporter()); final String uuid = "00000000-0000-0000-0000-000000000001"; @@ -795,7 +795,7 @@ public class TestPersistentProvenanceRepository { config.setMaxEventFileCapacity(1024L * 1024L); config.setSearchableFields(new ArrayList<>(SearchableFields.getStandardFields())); - repo = new PersistentProvenanceRepository(config); + repo = new PersistentProvenanceRepository(config, 75); repo.initialize(getEventReporter()); final String childId = "00000000-0000-0000-0000-000000000000"; @@ -845,7 +845,7 @@ public class TestPersistentProvenanceRepository { config.setMaxEventFileCapacity(1024L * 1024L); config.setSearchableFields(new ArrayList<>(SearchableFields.getStandardFields())); - repo = new PersistentProvenanceRepository(config); + repo = new PersistentProvenanceRepository(config, 75); repo.initialize(getEventReporter()); final String childId = "00000000-0000-0000-0000-000000000000"; @@ -892,7 +892,7 @@ public class TestPersistentProvenanceRepository { public void testCorrectProvenanceEventIdOnRestore() throws IOException { final RepositoryConfiguration config = createConfiguration(); config.setMaxEventFileLife(1, TimeUnit.SECONDS); - repo = new PersistentProvenanceRepository(config); + repo = new PersistentProvenanceRepository(config, 75); repo.initialize(getEventReporter()); final String uuid = "00000000-0000-0000-0000-000000000000"; @@ -918,7 +918,7 @@ public class TestPersistentProvenanceRepository { repo.close(); - final PersistentProvenanceRepository secondRepo = new PersistentProvenanceRepository(config); + final PersistentProvenanceRepository secondRepo = new PersistentProvenanceRepository(config, 75); secondRepo.initialize(getEventReporter()); final ProvenanceEventRecord event11 = builder.build(); @@ -938,7 +938,7 @@ public class TestPersistentProvenanceRepository { config.setMaxEventFileCapacity(1024L * 1024L); config.setSearchableFields(new ArrayList<>(SearchableFields.getStandardFields())); - repo = new PersistentProvenanceRepository(config); + repo = new PersistentProvenanceRepository(config, 75); repo.initialize(getEventReporter()); final String uuid = "00000000-0000-0000-0000-000000000000"; @@ -1002,7 +1002,7 @@ public class TestPersistentProvenanceRepository { final RepositoryConfiguration config = createConfiguration(); config.setMaxEventFileLife(500, TimeUnit.MILLISECONDS); config.setSearchableFields(new ArrayList<>(SearchableFields.getStandardFields())); - repo = new PersistentProvenanceRepository(config); + repo = new PersistentProvenanceRepository(config, 75); repo.initialize(getEventReporter()); final String uuid = "00000000-0000-0000-0000-000000000000"; @@ -1070,7 +1070,7 @@ public class TestPersistentProvenanceRepository { public void testMergeJournals() throws IOException, InterruptedException { final RepositoryConfiguration config = createConfiguration(); config.setMaxEventFileLife(3, TimeUnit.SECONDS); - repo = new PersistentProvenanceRepository(config); + repo = new PersistentProvenanceRepository(config, 75); repo.initialize(getEventReporter()); final Map attributes = new HashMap<>(); From dea9e224752d6fa2c7036174199b819e149ab0ed Mon Sep 17 00:00:00 2001 From: joewitt Date: Wed, 18 Mar 2015 16:27:01 -0400 Subject: [PATCH 028/116] NIFI-432 the short duration of 75ms appears too tight for present design. Moving to 2000 which is still very fast for testing. --- .../TestPersistentProvenanceRepository.java | 42 ++++++++++--------- 1 file changed, 22 insertions(+), 20 deletions(-) diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/test/java/org/apache/nifi/provenance/TestPersistentProvenanceRepository.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/test/java/org/apache/nifi/provenance/TestPersistentProvenanceRepository.java index 75e3514330..5be208bd4e 100644 --- a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/test/java/org/apache/nifi/provenance/TestPersistentProvenanceRepository.java +++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/test/java/org/apache/nifi/provenance/TestPersistentProvenanceRepository.java @@ -72,6 +72,8 @@ public class TestPersistentProvenanceRepository { public TestName name = new TestName(); private PersistentProvenanceRepository repo; + + public static final int DEFAULT_ROLLOVER_MILLIS = 2000; private RepositoryConfiguration createConfiguration() { final RepositoryConfiguration config = new RepositoryConfiguration(); @@ -170,7 +172,7 @@ public class TestPersistentProvenanceRepository { config.setCompressOnRollover(false); config.setJournalCount(10); config.setQueryThreadPoolSize(10); - repo = new PersistentProvenanceRepository(config, 75); + repo = new PersistentProvenanceRepository(config, DEFAULT_ROLLOVER_MILLIS); repo.initialize(getEventReporter()); final Map attributes = new HashMap<>(); @@ -218,7 +220,7 @@ public class TestPersistentProvenanceRepository { System.out.println("Closing and re-initializing"); repo.close(); - repo = new PersistentProvenanceRepository(config, 75); + repo = new PersistentProvenanceRepository(config, DEFAULT_ROLLOVER_MILLIS); repo.initialize(getEventReporter()); System.out.println("Re-initialized"); @@ -238,7 +240,7 @@ public class TestPersistentProvenanceRepository { final RepositoryConfiguration config = createConfiguration(); config.setMaxEventFileCapacity(1L); config.setMaxEventFileLife(1, TimeUnit.SECONDS); - repo = new PersistentProvenanceRepository(config, 75); + repo = new PersistentProvenanceRepository(config, DEFAULT_ROLLOVER_MILLIS); repo.initialize(getEventReporter()); final Map attributes = new HashMap<>(); @@ -262,7 +264,7 @@ public class TestPersistentProvenanceRepository { repo.close(); Thread.sleep(500L); // Give the repo time to shutdown (i.e., close all file handles, etc.) - repo = new PersistentProvenanceRepository(config, 75); + repo = new PersistentProvenanceRepository(config, DEFAULT_ROLLOVER_MILLIS); repo.initialize(getEventReporter()); final List recoveredRecords = repo.getEvents(0L, 12); @@ -285,7 +287,7 @@ public class TestPersistentProvenanceRepository { config.setMaxEventFileCapacity(1024L * 1024L); config.setMaxEventFileLife(2, TimeUnit.SECONDS); config.setSearchableFields(searchableFields); - repo = new PersistentProvenanceRepository(config, 75); + repo = new PersistentProvenanceRepository(config, DEFAULT_ROLLOVER_MILLIS); repo.initialize(getEventReporter()); final Map attributes = new HashMap<>(); @@ -339,7 +341,7 @@ public class TestPersistentProvenanceRepository { final RepositoryConfiguration config = createConfiguration(); config.setMaxEventFileLife(500, TimeUnit.MILLISECONDS); config.setSearchableFields(new ArrayList<>(SearchableFields.getStandardFields())); - repo = new PersistentProvenanceRepository(config, 75); + repo = new PersistentProvenanceRepository(config, DEFAULT_ROLLOVER_MILLIS); repo.initialize(getEventReporter()); final String uuid = "00000000-0000-0000-0000-000000000000"; @@ -383,7 +385,7 @@ public class TestPersistentProvenanceRepository { final RepositoryConfiguration config = createConfiguration(); config.setMaxEventFileLife(500, TimeUnit.MILLISECONDS); config.setCompressOnRollover(true); - repo = new PersistentProvenanceRepository(config, 75); + repo = new PersistentProvenanceRepository(config, DEFAULT_ROLLOVER_MILLIS); repo.initialize(getEventReporter()); final String uuid = "00000000-0000-0000-0000-000000000000"; @@ -421,7 +423,7 @@ public class TestPersistentProvenanceRepository { config.setMaxEventFileCapacity(1024L * 1024L); config.setSearchableFields(new ArrayList<>(SearchableFields.getStandardFields())); - repo = new PersistentProvenanceRepository(config, 75); + repo = new PersistentProvenanceRepository(config, DEFAULT_ROLLOVER_MILLIS); repo.initialize(getEventReporter()); final String uuid = "10000000-0000-0000-0000-000000000000"; @@ -480,7 +482,7 @@ public class TestPersistentProvenanceRepository { config.setMaxEventFileCapacity(1024L * 1024L); config.setSearchableFields(new ArrayList<>(SearchableFields.getStandardFields())); - repo = new PersistentProvenanceRepository(config, 75); + repo = new PersistentProvenanceRepository(config, DEFAULT_ROLLOVER_MILLIS); repo.initialize(getEventReporter()); final String uuid = "00000000-0000-0000-0000-000000000000"; @@ -543,7 +545,7 @@ public class TestPersistentProvenanceRepository { config.setMaxEventFileCapacity(1024L * 1024L); config.setSearchableFields(new ArrayList<>(SearchableFields.getStandardFields())); - repo = new PersistentProvenanceRepository(config, 75); + repo = new PersistentProvenanceRepository(config, DEFAULT_ROLLOVER_MILLIS); repo.initialize(getEventReporter()); final String uuid = "00000000-0000-0000-0000-000000000000"; @@ -625,7 +627,7 @@ public class TestPersistentProvenanceRepository { config.setMaxEventFileCapacity(1024L * 1024L); config.setSearchableFields(new ArrayList<>(SearchableFields.getStandardFields())); - repo = new PersistentProvenanceRepository(config, 75); + repo = new PersistentProvenanceRepository(config, DEFAULT_ROLLOVER_MILLIS); repo.initialize(getEventReporter()); final String uuid = "00000000-0000-0000-0000-000000000000"; @@ -681,7 +683,7 @@ public class TestPersistentProvenanceRepository { config.setMaxEventFileCapacity(1024L * 1024L); config.setSearchableFields(new ArrayList<>(SearchableFields.getStandardFields())); - repo = new PersistentProvenanceRepository(config, 75); + repo = new PersistentProvenanceRepository(config, DEFAULT_ROLLOVER_MILLIS); repo.initialize(getEventReporter()); final String uuid = "00000000-0000-0000-0000-000000000001"; @@ -736,7 +738,7 @@ public class TestPersistentProvenanceRepository { config.setMaxEventFileCapacity(1024L * 1024L); config.setSearchableFields(new ArrayList<>(SearchableFields.getStandardFields())); - repo = new PersistentProvenanceRepository(config, 75); + repo = new PersistentProvenanceRepository(config, DEFAULT_ROLLOVER_MILLIS); repo.initialize(getEventReporter()); final String uuid = "00000000-0000-0000-0000-000000000001"; @@ -795,7 +797,7 @@ public class TestPersistentProvenanceRepository { config.setMaxEventFileCapacity(1024L * 1024L); config.setSearchableFields(new ArrayList<>(SearchableFields.getStandardFields())); - repo = new PersistentProvenanceRepository(config, 75); + repo = new PersistentProvenanceRepository(config, DEFAULT_ROLLOVER_MILLIS); repo.initialize(getEventReporter()); final String childId = "00000000-0000-0000-0000-000000000000"; @@ -845,7 +847,7 @@ public class TestPersistentProvenanceRepository { config.setMaxEventFileCapacity(1024L * 1024L); config.setSearchableFields(new ArrayList<>(SearchableFields.getStandardFields())); - repo = new PersistentProvenanceRepository(config, 75); + repo = new PersistentProvenanceRepository(config, DEFAULT_ROLLOVER_MILLIS); repo.initialize(getEventReporter()); final String childId = "00000000-0000-0000-0000-000000000000"; @@ -892,7 +894,7 @@ public class TestPersistentProvenanceRepository { public void testCorrectProvenanceEventIdOnRestore() throws IOException { final RepositoryConfiguration config = createConfiguration(); config.setMaxEventFileLife(1, TimeUnit.SECONDS); - repo = new PersistentProvenanceRepository(config, 75); + repo = new PersistentProvenanceRepository(config, DEFAULT_ROLLOVER_MILLIS); repo.initialize(getEventReporter()); final String uuid = "00000000-0000-0000-0000-000000000000"; @@ -918,7 +920,7 @@ public class TestPersistentProvenanceRepository { repo.close(); - final PersistentProvenanceRepository secondRepo = new PersistentProvenanceRepository(config, 75); + final PersistentProvenanceRepository secondRepo = new PersistentProvenanceRepository(config, DEFAULT_ROLLOVER_MILLIS); secondRepo.initialize(getEventReporter()); final ProvenanceEventRecord event11 = builder.build(); @@ -938,7 +940,7 @@ public class TestPersistentProvenanceRepository { config.setMaxEventFileCapacity(1024L * 1024L); config.setSearchableFields(new ArrayList<>(SearchableFields.getStandardFields())); - repo = new PersistentProvenanceRepository(config, 75); + repo = new PersistentProvenanceRepository(config, DEFAULT_ROLLOVER_MILLIS); repo.initialize(getEventReporter()); final String uuid = "00000000-0000-0000-0000-000000000000"; @@ -1002,7 +1004,7 @@ public class TestPersistentProvenanceRepository { final RepositoryConfiguration config = createConfiguration(); config.setMaxEventFileLife(500, TimeUnit.MILLISECONDS); config.setSearchableFields(new ArrayList<>(SearchableFields.getStandardFields())); - repo = new PersistentProvenanceRepository(config, 75); + repo = new PersistentProvenanceRepository(config, DEFAULT_ROLLOVER_MILLIS); repo.initialize(getEventReporter()); final String uuid = "00000000-0000-0000-0000-000000000000"; @@ -1070,7 +1072,7 @@ public class TestPersistentProvenanceRepository { public void testMergeJournals() throws IOException, InterruptedException { final RepositoryConfiguration config = createConfiguration(); config.setMaxEventFileLife(3, TimeUnit.SECONDS); - repo = new PersistentProvenanceRepository(config, 75); + repo = new PersistentProvenanceRepository(config, DEFAULT_ROLLOVER_MILLIS); repo.initialize(getEventReporter()); final Map attributes = new HashMap<>(); From eb5ec703ba0d5c188822a37f6d7eed14af56a594 Mon Sep 17 00:00:00 2001 From: Oscar de la Pena Date: Thu, 19 Mar 2015 10:10:09 +0800 Subject: [PATCH 029/116] Fixes incorrect messages count in Provenance reporter. Adds Unit test to verify fix --- .../nifi/processors/kafka/PutKafka.java | 90 +++++++++---------- .../nifi/processors/kafka/TestPutKafka.java | 76 ++++++++++++++-- 2 files changed, 114 insertions(+), 52 deletions(-) diff --git a/nifi/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/java/org/apache/nifi/processors/kafka/PutKafka.java b/nifi/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/java/org/apache/nifi/processors/kafka/PutKafka.java index 4df4e08264..e0b7588c41 100644 --- a/nifi/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/java/org/apache/nifi/processors/kafka/PutKafka.java +++ b/nifi/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/java/org/apache/nifi/processors/kafka/PutKafka.java @@ -309,64 +309,61 @@ public class PutKafka extends AbstractProcessor { data = Arrays.copyOfRange(baos.getUnderlyingBuffer(), 0, baos.size() - delimiterBytes.length); } - createMessage: if ( data != null ) { + if ( data != null ) { // If the message has no data, ignore it. - if ( data.length == 0 ) { - data = null; - baos.reset(); - break createMessage; - } - - // either we ran out of data or we reached the end of the message. - // Either way, create the message because it's ready to send. - final KeyedMessage message; - if ( key == null ) { - message = new KeyedMessage<>(topic, data); - } else { - message = new KeyedMessage<>(topic, keyBytes, data); - } - - // Add the message to the list of messages ready to send. If we've reached our - // threshold of how many we're willing to send (or if we're out of data), go ahead - // and send the whole List. - messages.add(message); - messageBytes += data.length; - if ( messageBytes >= maxBufferSize || streamFinished ) { - // send the messages, then reset our state. - try { - producer.send(messages); - } catch (final Exception e) { - // we wrap the general exception in ProcessException because we want to separate - // failures in sending messages from general Exceptions that would indicate bugs - // in the Processor. Failure to send a message should be handled appropriately, but - // we don't want to catch the general Exception or RuntimeException in order to catch - // failures from Kafka's Producer. - throw new ProcessException("Failed to send messages to Kafka", e); + if ( data.length != 0 ) { + // either we ran out of data or we reached the end of the message. + // Either way, create the message because it's ready to send. + final KeyedMessage message; + if (key == null) { + message = new KeyedMessage<>(topic, data); + } else { + message = new KeyedMessage<>(topic, keyBytes, data); + } + + // Add the message to the list of messages ready to send. If we've reached our + // threshold of how many we're willing to send (or if we're out of data), go ahead + // and send the whole List. + messages.add(message); + messageBytes += data.length; + if (messageBytes >= maxBufferSize || streamFinished) { + // send the messages, then reset our state. + try { + producer.send(messages); + } catch (final Exception e) { + // we wrap the general exception in ProcessException because we want to separate + // failures in sending messages from general Exceptions that would indicate bugs + // in the Processor. Failure to send a message should be handled appropriately, but + // we don't want to catch the general Exception or RuntimeException in order to catch + // failures from Kafka's Producer. + throw new ProcessException("Failed to send messages to Kafka", e); + } + + messagesSent.addAndGet(messages.size()); // count number of messages sent + + // reset state + messages.clear(); + messageBytes = 0; + + // We've successfully sent a batch of messages. Keep track of the byte offset in the + // FlowFile of the last successfully sent message. This way, if the messages cannot + // all be successfully sent, we know where to split off the data. This allows us to then + // split off the first X number of bytes and send to 'success' and then split off the rest + // and send them to 'failure'. + lastMessageOffset.set(in.getBytesConsumed()); } - - messagesSent.addAndGet(messages.size()); // count number of messages sent - - // reset state - messages.clear(); - messageBytes = 0; - - // We've successfully sent a batch of messages. Keep track of the byte offset in the - // FlowFile of the last successfully sent message. This way, if the messages cannot - // all be successfully sent, we know where to split off the data. This allows us to then - // split off the first X number of bytes and send to 'success' and then split off the rest - // and send them to 'failure'. - lastMessageOffset.set(in.getBytesConsumed()); } - // reset BAOS so that we can start a new message. baos.reset(); data = null; + } } // If there are messages left, send them if ( !messages.isEmpty() ) { try { + messagesSent.addAndGet(messages.size()); // add count of messages producer.send(messages); } catch (final Exception e) { throw new ProcessException("Failed to send messages to Kafka", e); @@ -377,7 +374,6 @@ public class PutKafka extends AbstractProcessor { }); final long nanos = System.nanoTime() - start; - session.getProvenanceReporter().send(flowFile, "kafka://" + topic, "Sent " + messagesSent.get() + " messages"); session.transfer(flowFile, REL_SUCCESS); getLogger().info("Successfully sent {} messages to Kafka for {} in {} millis", new Object[] {messagesSent.get(), flowFile, TimeUnit.NANOSECONDS.toMillis(nanos)}); diff --git a/nifi/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/test/java/org/apache/nifi/processors/kafka/TestPutKafka.java b/nifi/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/test/java/org/apache/nifi/processors/kafka/TestPutKafka.java index b0f239451a..56a5c4b0ae 100644 --- a/nifi/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/test/java/org/apache/nifi/processors/kafka/TestPutKafka.java +++ b/nifi/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/test/java/org/apache/nifi/processors/kafka/TestPutKafka.java @@ -25,6 +25,7 @@ import java.util.Arrays; import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.concurrent.atomic.AtomicLong; import kafka.common.FailedToSendMessageException; import kafka.javaapi.producer.Producer; @@ -32,12 +33,14 @@ import kafka.producer.KeyedMessage; import kafka.producer.ProducerConfig; import org.apache.nifi.annotation.lifecycle.OnScheduled; + import org.apache.nifi.processor.ProcessContext; -import org.apache.nifi.util.MockFlowFile; -import org.apache.nifi.util.TestRunner; -import org.apache.nifi.util.TestRunners; +import org.apache.nifi.provenance.ProvenanceReporter; +import org.apache.nifi.util.*; import org.junit.Ignore; import org.junit.Test; +import org.mockito.Mockito; +import org.mockito.internal.util.reflection.Whitebox; public class TestPutKafka { @@ -139,8 +142,70 @@ public class TestPutKafka { assertTrue(Arrays.equals("3".getBytes(), msgs.get(2))); assertTrue(Arrays.equals("4".getBytes(), msgs.get(3))); } - - + + @Test + public void testProvenanceReporterMessagesCount(){ + final TestableProcessor processor = new TestableProcessor(); + + ProvenanceReporter spyProvenanceReporter = Mockito.spy(new MockProvenanceReporter()); + + AtomicLong idGenerator = new AtomicLong(0L); + SharedSessionState sharedState = new SharedSessionState(processor, idGenerator); + Whitebox.setInternalState(sharedState, "provenanceReporter", spyProvenanceReporter); + MockFlowFileQueue flowFileQueue = sharedState.getFlowFileQueue(); + MockSessionFactory sessionFactory = Mockito.mock(MockSessionFactory.class); + MockProcessSession mockProcessSession = new MockProcessSession(sharedState); + Mockito.when(sessionFactory.createSession()).thenReturn(mockProcessSession); + + + final TestRunner runner = TestRunners.newTestRunner(processor); + Whitebox.setInternalState(runner, "flowFileQueue", flowFileQueue); + Whitebox.setInternalState(runner, "sessionFactory", sessionFactory); + + runner.setProperty(PutKafka.TOPIC, "topic1"); + runner.setProperty(PutKafka.KEY, "key1"); + runner.setProperty(PutKafka.SEED_BROKERS, "localhost:1234"); + runner.setProperty(PutKafka.MESSAGE_DELIMITER, "\\n"); + + final byte[] bytes = "\n\n\n1\n2\n\n\n\n3\n4\n\n\n".getBytes(); + runner.enqueue(bytes); + runner.run(); + + MockFlowFile mockFlowFile = mockProcessSession.getFlowFilesForRelationship(PutKafka.REL_SUCCESS).get(0); + Mockito.verify(spyProvenanceReporter, Mockito.atLeastOnce()).send(mockFlowFile, "kafka://topic1", "Sent 4 messages"); + } + + @Test + public void testProvenanceReporterWithoutDelimiterMessagesCount(){ + final TestableProcessor processor = new TestableProcessor(); + + ProvenanceReporter spyProvenanceReporter = Mockito.spy(new MockProvenanceReporter()); + + AtomicLong idGenerator = new AtomicLong(0L); + SharedSessionState sharedState = new SharedSessionState(processor, idGenerator); + Whitebox.setInternalState(sharedState, "provenanceReporter", spyProvenanceReporter); + MockFlowFileQueue flowFileQueue = sharedState.getFlowFileQueue(); + MockSessionFactory sessionFactory = Mockito.mock(MockSessionFactory.class); + MockProcessSession mockProcessSession = new MockProcessSession(sharedState); + Mockito.when(sessionFactory.createSession()).thenReturn(mockProcessSession); + + + final TestRunner runner = TestRunners.newTestRunner(processor); + Whitebox.setInternalState(runner, "flowFileQueue", flowFileQueue); + Whitebox.setInternalState(runner, "sessionFactory", sessionFactory); + + runner.setProperty(PutKafka.TOPIC, "topic1"); + runner.setProperty(PutKafka.KEY, "key1"); + runner.setProperty(PutKafka.SEED_BROKERS, "localhost:1234"); + + final byte[] bytes = "\n\n\n1\n2\n\n\n\n3\n4\n\n\n".getBytes(); + runner.enqueue(bytes); + runner.run(); + + MockFlowFile mockFlowFile = mockProcessSession.getFlowFilesForRelationship(PutKafka.REL_SUCCESS).get(0); + Mockito.verify(spyProvenanceReporter, Mockito.atLeastOnce()).send(mockFlowFile, "kafka://topic1"); + } + @Test @Ignore("Intended only for local testing; requires an actual running instance of Kafka & ZooKeeper...") public void testKeyValuePut() { @@ -233,4 +298,5 @@ public class TestPutKafka { failAfter = successCount; } } + } From 7272d0df58c23d099809bf96993b55d73c617476 Mon Sep 17 00:00:00 2001 From: Bobby Owolabi Date: Wed, 18 Mar 2015 23:30:57 -0400 Subject: [PATCH 030/116] NIFI-396 created tests to demonstrate the situations where the ProcessSession throws an Exception and where it doesn't after it returns from the callback --- .../TestStandardProcessSession.java | 108 ++++++++++++++++++ 1 file changed, 108 insertions(+) diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/TestStandardProcessSession.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/TestStandardProcessSession.java index 1ff63c545b..2d09ea5b89 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/TestStandardProcessSession.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/TestStandardProcessSession.java @@ -56,6 +56,7 @@ import org.apache.nifi.controller.repository.claim.StandardContentClaimManager; import org.apache.nifi.flowfile.FlowFile; import org.apache.nifi.groups.ProcessGroup; import org.apache.nifi.processor.Relationship; +import org.apache.nifi.processor.exception.FlowFileAccessException; import org.apache.nifi.processor.exception.MissingFlowFileException; import org.apache.nifi.processor.exception.ProcessException; import org.apache.nifi.processor.io.InputStreamCallback; @@ -65,6 +66,8 @@ import org.apache.nifi.provenance.MockProvenanceEventRepository; import org.apache.nifi.provenance.ProvenanceEventRecord; import org.apache.nifi.provenance.ProvenanceEventRepository; import org.apache.nifi.provenance.ProvenanceEventType; +import org.apache.nifi.stream.io.ByteArrayOutputStream; +import org.apache.nifi.util.ObjectHolder; import org.junit.After; import org.junit.Assert; import org.junit.Before; @@ -235,6 +238,105 @@ public class TestStandardProcessSession { assertEquals(0, contentRepo.getExistingClaims().size()); } + @Test(expected = FlowFileAccessException.class) + public void testAppendAfterSessionClosesStream() throws IOException { + final ContentClaim claim = contentRepo.create(false); + final FlowFileRecord flowFileRecord = new StandardFlowFileRecord.Builder() + .contentClaim(claim) + .addAttribute("uuid", "12345678-1234-1234-1234-123456789012") + .entryDate(System.currentTimeMillis()) + .build(); + flowFileQueue.put(flowFileRecord); + FlowFile flowFile = session.get(); + assertNotNull(flowFile); + final ObjectHolder outputStreamHolder = new ObjectHolder<>(null); + flowFile = session.append(flowFile, new OutputStreamCallback() { + @Override + public void process(final OutputStream outputStream) throws IOException { + outputStreamHolder.set(outputStream); + } + }); + try (final OutputStream outputStream = outputStreamHolder.get()) { + outputStream.write(5); + } + } + + @Test(expected = FlowFileAccessException.class) + public void testReadAfterSessionClosesStream() throws IOException { + final ContentClaim claim = contentRepo.create(false); + final FlowFileRecord flowFileRecord = new StandardFlowFileRecord.Builder() + .contentClaim(claim) + .addAttribute("uuid", "12345678-1234-1234-1234-123456789012") + .entryDate(System.currentTimeMillis()) + .build(); + flowFileQueue.put(flowFileRecord); + FlowFile flowFile = session.get(); + assertNotNull(flowFile); + final ObjectHolder inputStreamHolder = new ObjectHolder<>(null); + session.read(flowFile, new InputStreamCallback() { + @Override + public void process(final InputStream inputStream) throws IOException { + inputStreamHolder.set(inputStream); + } + }); + try (final InputStream inputStream = inputStreamHolder.get()) { + inputStream.read(); + } + } + + @Test + public void testStreamAfterSessionClosesStream() throws IOException { + final ContentClaim claim = contentRepo.create(false); + final FlowFileRecord flowFileRecord = new StandardFlowFileRecord.Builder() + .contentClaim(claim) + .addAttribute("uuid", "12345678-1234-1234-1234-123456789012") + .entryDate(System.currentTimeMillis()) + .build(); + flowFileQueue.put(flowFileRecord); + FlowFile flowFile = session.get(); + assertNotNull(flowFile); + final ObjectHolder inputStreamHolder = new ObjectHolder<>(null); + final ObjectHolder outputStreamHolder = new ObjectHolder<>(null); + flowFile = session.write(flowFile, new StreamCallback() { + @Override + public void process(final InputStream input, final OutputStream output) throws IOException { + inputStreamHolder.set(input); + outputStreamHolder.set(output); + } + }); + try (final InputStream inputStream = inputStreamHolder.get()) { + inputStream.read(); + Assert.fail("Expected Exception to be thrown when read is attempted after session closes stream"); + } catch (final Exception ex) {} + try (final OutputStream outputStream = outputStreamHolder.get()) { + outputStream.write(5); + Assert.fail("Expected Exception to be thrown when write is attempted after session closes stream"); + } catch (final Exception ex) {} + } + + @Test(expected = FlowFileAccessException.class) + public void testWriteAfterSessionClosesStream() throws IOException { + final ContentClaim claim = contentRepo.create(false); + final FlowFileRecord flowFileRecord = new StandardFlowFileRecord.Builder() + .contentClaim(claim) + .addAttribute("uuid", "12345678-1234-1234-1234-123456789012") + .entryDate(System.currentTimeMillis()) + .build(); + flowFileQueue.put(flowFileRecord); + FlowFile flowFile = session.get(); + assertNotNull(flowFile); + final ObjectHolder outputStreamHolder = new ObjectHolder<>(null); + flowFile = session.write(flowFile, new OutputStreamCallback() { + @Override + public void process(final OutputStream out) throws IOException { + outputStreamHolder.set(out); + } + }); + try (final OutputStream outputStream = outputStreamHolder.get()) { + outputStream.write(5); + } + } + @Test public void testCreateThenRollbackRemovesContent() throws IOException { @@ -998,6 +1100,12 @@ public class TestStandardProcessSession { public ContentClaim create(boolean lossTolerant) throws IOException { final ContentClaim claim = claimManager.newContentClaim("container", "section", String.valueOf(idGenerator.getAndIncrement()), false); claimantCounts.put(claim, new AtomicInteger(1)); + final Path path = getPath(claim); + final Path parent = path.getParent(); + if (Files.exists(parent) == false) { + Files.createDirectories(parent); + } + Files.createFile(getPath(claim)); return claim; } From e2760f8c980583d285137134e05c435c930fb4d2 Mon Sep 17 00:00:00 2001 From: Bobby Owolabi Date: Thu, 19 Mar 2015 00:54:24 -0400 Subject: [PATCH 031/116] NIFI-396 added a DisableOnCloseInputStream class; modified StandardProcessSession to prevent access of the Input/OutputStreams after callbacks have been executed; updated tests --- .../repository/StandardProcessSession.java | 11 +- .../io/DisableOnCloseInputStream.java | 93 ++++++ .../TestStandardProcessSession.java | 295 ++++++++++-------- 3 files changed, 266 insertions(+), 133 deletions(-) create mode 100644 nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/io/DisableOnCloseInputStream.java diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/StandardProcessSession.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/StandardProcessSession.java index 8d2e4567ef..e5cd03ea19 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/StandardProcessSession.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/StandardProcessSession.java @@ -46,6 +46,7 @@ import org.apache.nifi.controller.ProcessorNode; import org.apache.nifi.controller.repository.claim.ContentClaim; import org.apache.nifi.controller.repository.io.ByteCountingInputStream; import org.apache.nifi.controller.repository.io.ByteCountingOutputStream; +import org.apache.nifi.controller.repository.io.DisableOnCloseInputStream; import org.apache.nifi.controller.repository.io.DisableOnCloseOutputStream; import org.apache.nifi.controller.repository.io.FlowFileAccessInputStream; import org.apache.nifi.controller.repository.io.FlowFileAccessOutputStream; @@ -1735,7 +1736,8 @@ public final class StandardProcessSession implements ProcessSession, ProvenanceE try (final InputStream rawIn = getInputStream(source, record.getCurrentClaim(), record.getCurrentClaimOffset()); final InputStream limitedIn = new LimitedInputStream(rawIn, source.getSize()); - final ByteCountingInputStream countingStream = new ByteCountingInputStream(limitedIn, this.bytesRead)) { + final InputStream disableOnCloseIn = new DisableOnCloseInputStream(limitedIn); + final ByteCountingInputStream countingStream = new ByteCountingInputStream(disableOnCloseIn, this.bytesRead)) { // We want to differentiate between IOExceptions thrown by the repository and IOExceptions thrown from // Processor code. As a result, as have the FlowFileAccessInputStream that catches IOException from the repository @@ -2180,9 +2182,10 @@ public final class StandardProcessSession implements ProcessSession, ProvenanceE try (final InputStream rawIn = getInputStream(source, currClaim, record.getCurrentClaimOffset()); final InputStream limitedIn = new LimitedInputStream(rawIn, source.getSize()); - final InputStream countingIn = new ByteCountingInputStream(limitedIn, bytesRead); - final OutputStream disableOnClose = new DisableOnCloseOutputStream(currentWriteClaimStream); - final OutputStream countingOut = new ByteCountingOutputStream(disableOnClose, writtenHolder)) { + final InputStream disableOnCloseIn = new DisableOnCloseInputStream(limitedIn); + final InputStream countingIn = new ByteCountingInputStream(disableOnCloseIn, bytesRead); + final OutputStream disableOnCloseOut = new DisableOnCloseOutputStream(currentWriteClaimStream); + final OutputStream countingOut = new ByteCountingOutputStream(disableOnCloseOut, writtenHolder)) { recursionSet.add(source); diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/io/DisableOnCloseInputStream.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/io/DisableOnCloseInputStream.java new file mode 100644 index 0000000000..ddcf6c9bf0 --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/io/DisableOnCloseInputStream.java @@ -0,0 +1,93 @@ +/* + * 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.controller.repository.io; + +import java.io.IOException; +import java.io.InputStream; + +/** + * Wraps an existing InputStream, so that when {@link InputStream#close()} is + * called, the underlying InputStream is NOT closed but this InputStream can no + * longer be written to + */ +public class DisableOnCloseInputStream extends InputStream { + + private final InputStream wrapped; + private boolean closed = false; + + public DisableOnCloseInputStream(final InputStream wrapped) { + this.wrapped = wrapped; + } + + @Override + public int read() throws IOException { + checkClosed(); + return wrapped.read(); + } + + @Override + public int read(byte[] b) throws IOException { + checkClosed(); + return wrapped.read(b); + } + + @Override + public int read(byte[] b, int off, int len) throws IOException { + checkClosed(); + return wrapped.read(b, off, len); + } + + @Override + public long skip(long n) throws IOException { + checkClosed(); + return wrapped.skip(n); + } + + @Override + public int available() throws IOException { + return wrapped.available(); + } + + private void checkClosed() throws IOException { + if (closed) { + throw new IOException("Stream is closed"); + } + } + + @Override + public void close() throws IOException { + closed = true; + } + + @Override + public void mark(int readlimit) { + if (closed == false) { + wrapped.mark(readlimit); + } + } + + @Override + public synchronized void reset() throws IOException { + checkClosed(); + wrapped.reset(); + } + + @Override + public boolean markSupported() { + return wrapped.markSupported(); + } +} diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/TestStandardProcessSession.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/TestStandardProcessSession.java index 2d09ea5b89..ef2fb935d5 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/TestStandardProcessSession.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/TestStandardProcessSession.java @@ -66,7 +66,6 @@ import org.apache.nifi.provenance.MockProvenanceEventRepository; import org.apache.nifi.provenance.ProvenanceEventRecord; import org.apache.nifi.provenance.ProvenanceEventRepository; import org.apache.nifi.provenance.ProvenanceEventType; -import org.apache.nifi.stream.io.ByteArrayOutputStream; import org.apache.nifi.util.ObjectHolder; import org.junit.After; import org.junit.Assert; @@ -238,7 +237,61 @@ public class TestStandardProcessSession { assertEquals(0, contentRepo.getExistingClaims().size()); } - @Test(expected = FlowFileAccessException.class) + private void assertDisabled(final OutputStream outputStream) { + try { + outputStream.write(new byte[0]); + Assert.fail("Expected OutputStream to be disabled; was able to call write(byte[])"); + } catch (final Exception ex) { + Assert.assertEquals(FlowFileAccessException.class, ex.getClass()); + } + try { + outputStream.write(0); + Assert.fail("Expected OutputStream to be disabled; was able to call write(int)"); + } catch (final Exception ex) { + Assert.assertEquals(FlowFileAccessException.class, ex.getClass()); + } + try { + outputStream.write(new byte[0], 0, 0); + Assert.fail("Expected OutputStream to be disabled; was able to call write(byte[], int, int)"); + } catch (final Exception ex) { + Assert.assertEquals(FlowFileAccessException.class, ex.getClass()); + } + } + + private void assertDisabled(final InputStream inputStream) { + try { + inputStream.read(); + Assert.fail("Expected InputStream to be disabled; was able to call read()"); + } catch (final Exception ex) { + Assert.assertEquals(FlowFileAccessException.class, ex.getClass()); + } + try { + inputStream.read(new byte[0]); + Assert.fail("Expected InputStream to be disabled; was able to call read(byte[])"); + } catch (final Exception ex) { + Assert.assertEquals(FlowFileAccessException.class, ex.getClass()); + } + try { + inputStream.read(new byte[0], 0, 0); + Assert.fail("Expected InputStream to be disabled; was able to call read(byte[], int, int)"); + } catch (final Exception ex) { + Assert.assertEquals(FlowFileAccessException.class, ex.getClass()); + } + try { + inputStream.reset(); + Assert.fail("Expected InputStream to be disabled; was able to call reset()"); + } catch (final Exception ex) { + Assert.assertEquals(FlowFileAccessException.class, ex.getClass()); + } + try { + inputStream.skip(1L); + Assert.fail("Expected InputStream to be disabled; was able to call skip(long)"); + } catch (final Exception ex) { + Assert.assertEquals(FlowFileAccessException.class, ex.getClass()); + } + } + + @Test public void testAppendAfterSessionClosesStream() throws IOException { final ContentClaim claim = contentRepo.create(false); final FlowFileRecord flowFileRecord = new StandardFlowFileRecord.Builder() @@ -256,12 +309,10 @@ public class TestStandardProcessSession { outputStreamHolder.set(outputStream); } }); - try (final OutputStream outputStream = outputStreamHolder.get()) { - outputStream.write(5); - } + assertDisabled(outputStreamHolder.get()); } - @Test(expected = FlowFileAccessException.class) + @Test public void testReadAfterSessionClosesStream() throws IOException { final ContentClaim claim = contentRepo.create(false); final FlowFileRecord flowFileRecord = new StandardFlowFileRecord.Builder() @@ -279,9 +330,7 @@ public class TestStandardProcessSession { inputStreamHolder.set(inputStream); } }); - try (final InputStream inputStream = inputStreamHolder.get()) { - inputStream.read(); - } + assertDisabled(inputStreamHolder.get()); } @Test @@ -304,17 +353,11 @@ public class TestStandardProcessSession { outputStreamHolder.set(output); } }); - try (final InputStream inputStream = inputStreamHolder.get()) { - inputStream.read(); - Assert.fail("Expected Exception to be thrown when read is attempted after session closes stream"); - } catch (final Exception ex) {} - try (final OutputStream outputStream = outputStreamHolder.get()) { - outputStream.write(5); - Assert.fail("Expected Exception to be thrown when write is attempted after session closes stream"); - } catch (final Exception ex) {} - } + assertDisabled(inputStreamHolder.get()); + assertDisabled(outputStreamHolder.get()); + } - @Test(expected = FlowFileAccessException.class) + @Test public void testWriteAfterSessionClosesStream() throws IOException { final ContentClaim claim = contentRepo.create(false); final FlowFileRecord flowFileRecord = new StandardFlowFileRecord.Builder() @@ -332,9 +375,7 @@ public class TestStandardProcessSession { outputStreamHolder.set(out); } }); - try (final OutputStream outputStream = outputStreamHolder.get()) { - outputStream.write(5); - } + assertDisabled(outputStreamHolder.get()); } @Test @@ -385,7 +426,6 @@ public class TestStandardProcessSession { assertEquals(0, provenanceRepo.getEvents(0L, 100000).size()); } - @Test public void testProvenanceEventsEmittedForForkIfNotRemoved() throws IOException { final FlowFileRecord flowFileRecord = new StandardFlowFileRecord.Builder() @@ -425,59 +465,59 @@ public class TestStandardProcessSession { @Test public void testUpdateAttributesThenJoin() throws IOException { final FlowFileRecord flowFileRecord1 = new StandardFlowFileRecord.Builder() - .id(1L) - .addAttribute("uuid", "11111111-1111-1111-1111-111111111111") - .entryDate(System.currentTimeMillis()) - .build(); - + .id(1L) + .addAttribute("uuid", "11111111-1111-1111-1111-111111111111") + .entryDate(System.currentTimeMillis()) + .build(); + final FlowFileRecord flowFileRecord2 = new StandardFlowFileRecord.Builder() - .id(2L) - .addAttribute("uuid", "22222222-2222-2222-2222-222222222222") - .entryDate(System.currentTimeMillis()) - .build(); - + .id(2L) + .addAttribute("uuid", "22222222-2222-2222-2222-222222222222") + .entryDate(System.currentTimeMillis()) + .build(); + flowFileQueue.put(flowFileRecord1); flowFileQueue.put(flowFileRecord2); - + FlowFile ff1 = session.get(); FlowFile ff2 = session.get(); ff1 = session.putAttribute(ff1, "index", "1"); ff2 = session.putAttribute(ff2, "index", "2"); - + final List parents = new ArrayList<>(2); parents.add(ff1); parents.add(ff2); - + final FlowFile child = session.create(parents); - + final Relationship rel = new Relationship.Builder().name("A").build(); - + session.transfer(ff1, rel); session.transfer(ff2, rel); session.transfer(child, rel); - + session.commit(); - + final List events = provenanceRepo.getEvents(0L, 1000); // We should have a JOIN and 2 ATTRIBUTE_MODIFIED's assertEquals(3, events.size()); - + int joinCount = 0; int ff1UpdateCount = 0; int ff2UpdateCount = 0; - - for ( final ProvenanceEventRecord event : events ) { + + for (final ProvenanceEventRecord event : events) { switch (event.getEventType()) { case JOIN: assertEquals(child.getAttribute("uuid"), event.getFlowFileUuid()); joinCount++; break; case ATTRIBUTES_MODIFIED: - if ( event.getFlowFileUuid().equals(ff1.getAttribute("uuid")) ) { + if (event.getFlowFileUuid().equals(ff1.getAttribute("uuid"))) { ff1UpdateCount++; - } else if ( event.getFlowFileUuid().equals(ff2.getAttribute("uuid")) ) { + } else if (event.getFlowFileUuid().equals(ff2.getAttribute("uuid"))) { ff2UpdateCount++; } else { Assert.fail("Got ATTRIBUTE_MODIFIED for wrong FlowFile: " + event.getFlowFileUuid()); @@ -487,14 +527,14 @@ public class TestStandardProcessSession { Assert.fail("Unexpected event type: " + event); } } - + assertEquals(1, joinCount); assertEquals(1, ff1UpdateCount); assertEquals(1, ff2UpdateCount); - + assertEquals(1, joinCount); } - + @Test public void testForkOneToOneReported() throws IOException { final FlowFileRecord flowFileRecord = new StandardFlowFileRecord.Builder() @@ -804,34 +844,34 @@ public class TestStandardProcessSession { @Test public void testContentNotFoundExceptionThrownWhenUnableToReadDataOffsetTooLarge() { final FlowFileRecord flowFileRecord = new StandardFlowFileRecord.Builder() - .addAttribute("uuid", "12345678-1234-1234-1234-123456789012") - .entryDate(System.currentTimeMillis()) - .contentClaim(new ContentClaim() { - @Override - public int compareTo(ContentClaim arg0) { - return 0; - } - - @Override - public String getId() { - return "0"; - } - - @Override - public String getContainer() { - return "container"; - } - - @Override - public String getSection() { - return "section"; - } - - @Override - public boolean isLossTolerant() { - return true; - } - }).build(); + .addAttribute("uuid", "12345678-1234-1234-1234-123456789012") + .entryDate(System.currentTimeMillis()) + .contentClaim(new ContentClaim() { + @Override + public int compareTo(ContentClaim arg0) { + return 0; + } + + @Override + public String getId() { + return "0"; + } + + @Override + public String getContainer() { + return "container"; + } + + @Override + public String getSection() { + return "section"; + } + + @Override + public boolean isLossTolerant() { + return true; + } + }).build(); flowFileQueue.put(flowFileRecord); FlowFile ff1 = session.get(); @@ -844,35 +884,35 @@ public class TestStandardProcessSession { session.commit(); final FlowFileRecord flowFileRecord2 = new StandardFlowFileRecord.Builder() - .addAttribute("uuid", "12345678-1234-1234-1234-123456789012") - .entryDate(System.currentTimeMillis()) - .contentClaim(new ContentClaim() { - @Override - public int compareTo(ContentClaim arg0) { - return 0; - } - - @Override - public String getId() { - return "0"; - } - - @Override - public String getContainer() { - return "container"; - } - - @Override - public String getSection() { - return "section"; - } - - @Override - public boolean isLossTolerant() { - return true; - } - }) - .contentClaimOffset(1000L).size(1L).build(); + .addAttribute("uuid", "12345678-1234-1234-1234-123456789012") + .entryDate(System.currentTimeMillis()) + .contentClaim(new ContentClaim() { + @Override + public int compareTo(ContentClaim arg0) { + return 0; + } + + @Override + public String getId() { + return "0"; + } + + @Override + public String getContainer() { + return "container"; + } + + @Override + public String getSection() { + return "section"; + } + + @Override + public boolean isLossTolerant() { + return true; + } + }) + .contentClaimOffset(1000L).size(1L).build(); flowFileQueue.put(flowFileRecord2); // attempt to read the data. @@ -933,21 +973,20 @@ public class TestStandardProcessSession { } } - @Test public void testCreateEmitted() throws IOException { FlowFile newFlowFile = session.create(); session.transfer(newFlowFile, new Relationship.Builder().name("A").build()); session.commit(); - + final List events = provenanceRepo.getEvents(0L, 10000); assertFalse(events.isEmpty()); assertEquals(1, events.size()); - + final ProvenanceEventRecord event = events.get(0); assertEquals(ProvenanceEventType.CREATE, event.getEventType()); } - + @Test public void testContentModifiedNotEmittedForCreate() throws IOException { FlowFile newFlowFile = session.create(); @@ -958,23 +997,23 @@ public class TestStandardProcessSession { }); session.transfer(newFlowFile, new Relationship.Builder().name("A").build()); session.commit(); - + final List events = provenanceRepo.getEvents(0L, 10000); assertFalse(events.isEmpty()); assertEquals(1, events.size()); - + final ProvenanceEventRecord event = events.get(0); assertEquals(ProvenanceEventType.CREATE, event.getEventType()); } - + @Test public void testContentModifiedEmittedAndNotAttributesModified() throws IOException { final FlowFileRecord flowFile = new StandardFlowFileRecord.Builder() - .id(1L) - .addAttribute("uuid", "000000000000-0000-0000-0000-00000000") - .build(); + .id(1L) + .addAttribute("uuid", "000000000000-0000-0000-0000-00000000") + .build(); this.flowFileQueue.put(flowFile); - + FlowFile existingFlowFile = session.get(); existingFlowFile = session.write(existingFlowFile, new OutputStreamCallback() { @Override @@ -984,38 +1023,36 @@ public class TestStandardProcessSession { existingFlowFile = session.putAttribute(existingFlowFile, "attr", "a"); session.transfer(existingFlowFile, new Relationship.Builder().name("A").build()); session.commit(); - + final List events = provenanceRepo.getEvents(0L, 10000); assertFalse(events.isEmpty()); assertEquals(1, events.size()); - + final ProvenanceEventRecord event = events.get(0); assertEquals(ProvenanceEventType.CONTENT_MODIFIED, event.getEventType()); } - + @Test public void testAttributesModifiedEmitted() throws IOException { final FlowFileRecord flowFile = new StandardFlowFileRecord.Builder() - .id(1L) - .addAttribute("uuid", "000000000000-0000-0000-0000-00000000") - .build(); + .id(1L) + .addAttribute("uuid", "000000000000-0000-0000-0000-00000000") + .build(); this.flowFileQueue.put(flowFile); - + FlowFile existingFlowFile = session.get(); existingFlowFile = session.putAttribute(existingFlowFile, "attr", "a"); session.transfer(existingFlowFile, new Relationship.Builder().name("A").build()); session.commit(); - + final List events = provenanceRepo.getEvents(0L, 10000); assertFalse(events.isEmpty()); assertEquals(1, events.size()); - + final ProvenanceEventRecord event = events.get(0); assertEquals(ProvenanceEventType.ATTRIBUTES_MODIFIED, event.getEventType()); } - - - + private static class MockFlowFileRepository implements FlowFileRepository { private final AtomicLong idGenerator = new AtomicLong(0L); @@ -1082,7 +1119,7 @@ public class TestStandardProcessSession { @Override public void shutdown() { } - + public Set getExistingClaims() { final Set claims = new HashSet<>(); From cd183be4410bdc88de7d3a0026452ab62eb10621 Mon Sep 17 00:00:00 2001 From: Bobby Owolabi Date: Thu, 19 Mar 2015 01:18:22 -0400 Subject: [PATCH 032/116] NIFI-396 updated the javadocs of ProcessSession to reflect that a FlowFileAccessExcpetion will be thrown if an Input/Output Stream is attempted to be accessed after the callback is executed. --- .../apache/nifi/processor/ProcessSession.java | 40 ++++++++++++++----- 1 file changed, 31 insertions(+), 9 deletions(-) diff --git a/nifi/nifi-api/src/main/java/org/apache/nifi/processor/ProcessSession.java b/nifi/nifi-api/src/main/java/org/apache/nifi/processor/ProcessSession.java index d3de916572..7b855f2833 100644 --- a/nifi/nifi-api/src/main/java/org/apache/nifi/processor/ProcessSession.java +++ b/nifi/nifi-api/src/main/java/org/apache/nifi/processor/ProcessSession.java @@ -484,6 +484,9 @@ public interface ProcessSession { /** * Executes the given callback against the contents corresponding to the * given FlowFile. + * + * Note: The OutputStream provided to the given OutputStreamCallback + * will not be accessible once this method has completed its execution. * * @param source * @param reader @@ -498,9 +501,11 @@ public interface ProcessSession { * destroyed, and the session is automatically rolled back and what is left * of the FlowFile is destroyed. * @throws FlowFileAccessException if some IO problem occurs accessing - * FlowFile content + * FlowFile content; if an attempt is made to access the InputStream + * provided to the given InputStreamCallback after this method completed its + * execution */ - void read(FlowFile source, InputStreamCallback reader); + void read(FlowFile source, InputStreamCallback reader) throws FlowFileAccessException; /** * Combines the content of all given source FlowFiles into a single given @@ -560,7 +565,10 @@ public interface ProcessSession { /** * Executes the given callback against the content corresponding to the - * given FlowFile + * given FlowFile. + * + * Note: The OutputStream provided to the given OutputStreamCallback + * will not be accessible once this method has completed its execution. * * @param source * @param writer @@ -576,13 +584,19 @@ public interface ProcessSession { * destroyed, and the session is automatically rolled back and what is left * of the FlowFile is destroyed. * @throws FlowFileAccessException if some IO problem occurs accessing - * FlowFile content + * FlowFile content; if an attempt is made to access the OutputStream + * provided to the given OutputStreamCallaback after this method completed + * its execution */ - FlowFile write(FlowFile source, OutputStreamCallback writer); + FlowFile write(FlowFile source, OutputStreamCallback writer) throws FlowFileAccessException; /** * Executes the given callback against the content corresponding to the - * given flow file + * given flow file. + * + * Note: The InputStream & OutputStream provided to the given + * StreamCallback will not be accessible once this method has completed its + * execution. * * @param source * @param writer @@ -598,20 +612,28 @@ public interface ProcessSession { * destroyed, and the session is automatically rolled back and what is left * of the FlowFile is destroyed. * @throws FlowFileAccessException if some IO problem occurs accessing - * FlowFile content + * FlowFile content; if an attempt is made to access the InputStream or + * OutputStream provided to the given StreamCallback after this method + * completed its execution */ - FlowFile write(FlowFile source, StreamCallback writer); + FlowFile write(FlowFile source, StreamCallback writer) throws FlowFileAccessException; /** * Executes the given callback against the content corresponding to the * given FlowFile, such that any data written to the OutputStream of the * content will be appended to the end of FlowFile. + * + * Note: The OutputStream provided to the given OutputStreamCallback + * will not be accessible once this method has completed its execution. * * @param source * @param writer * @return + * @throws FlowFileAccessException if an attempt is made to access the + * OutputStream provided to the given OutputStreamCallaback after this method + * completed its execution */ - FlowFile append(FlowFile source, OutputStreamCallback writer); + FlowFile append(FlowFile source, OutputStreamCallback writer) throws FlowFileAccessException; /** * Writes to the given FlowFile all content from the given content path. From 3f3623647367421cd8eb318668144aca1afb9bf4 Mon Sep 17 00:00:00 2001 From: Bobby Owolabi Date: Thu, 19 Mar 2015 01:43:17 -0400 Subject: [PATCH 033/116] NIFI-396 reverting accidentially modified whitespace in TestStandardProcessSession --- .../TestStandardProcessSession.java | 212 +++++++++--------- 1 file changed, 108 insertions(+), 104 deletions(-) diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/TestStandardProcessSession.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/TestStandardProcessSession.java index ef2fb935d5..ca68725bf1 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/TestStandardProcessSession.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/TestStandardProcessSession.java @@ -426,6 +426,7 @@ public class TestStandardProcessSession { assertEquals(0, provenanceRepo.getEvents(0L, 100000).size()); } + @Test public void testProvenanceEventsEmittedForForkIfNotRemoved() throws IOException { final FlowFileRecord flowFileRecord = new StandardFlowFileRecord.Builder() @@ -465,59 +466,59 @@ public class TestStandardProcessSession { @Test public void testUpdateAttributesThenJoin() throws IOException { final FlowFileRecord flowFileRecord1 = new StandardFlowFileRecord.Builder() - .id(1L) - .addAttribute("uuid", "11111111-1111-1111-1111-111111111111") - .entryDate(System.currentTimeMillis()) - .build(); - + .id(1L) + .addAttribute("uuid", "11111111-1111-1111-1111-111111111111") + .entryDate(System.currentTimeMillis()) + .build(); + final FlowFileRecord flowFileRecord2 = new StandardFlowFileRecord.Builder() - .id(2L) - .addAttribute("uuid", "22222222-2222-2222-2222-222222222222") - .entryDate(System.currentTimeMillis()) - .build(); - + .id(2L) + .addAttribute("uuid", "22222222-2222-2222-2222-222222222222") + .entryDate(System.currentTimeMillis()) + .build(); + flowFileQueue.put(flowFileRecord1); flowFileQueue.put(flowFileRecord2); - + FlowFile ff1 = session.get(); FlowFile ff2 = session.get(); ff1 = session.putAttribute(ff1, "index", "1"); ff2 = session.putAttribute(ff2, "index", "2"); - + final List parents = new ArrayList<>(2); parents.add(ff1); parents.add(ff2); - + final FlowFile child = session.create(parents); - + final Relationship rel = new Relationship.Builder().name("A").build(); - + session.transfer(ff1, rel); session.transfer(ff2, rel); session.transfer(child, rel); - + session.commit(); - + final List events = provenanceRepo.getEvents(0L, 1000); // We should have a JOIN and 2 ATTRIBUTE_MODIFIED's assertEquals(3, events.size()); - + int joinCount = 0; int ff1UpdateCount = 0; int ff2UpdateCount = 0; - - for (final ProvenanceEventRecord event : events) { + + for ( final ProvenanceEventRecord event : events ) { switch (event.getEventType()) { case JOIN: assertEquals(child.getAttribute("uuid"), event.getFlowFileUuid()); joinCount++; break; case ATTRIBUTES_MODIFIED: - if (event.getFlowFileUuid().equals(ff1.getAttribute("uuid"))) { + if ( event.getFlowFileUuid().equals(ff1.getAttribute("uuid")) ) { ff1UpdateCount++; - } else if (event.getFlowFileUuid().equals(ff2.getAttribute("uuid"))) { + } else if ( event.getFlowFileUuid().equals(ff2.getAttribute("uuid")) ) { ff2UpdateCount++; } else { Assert.fail("Got ATTRIBUTE_MODIFIED for wrong FlowFile: " + event.getFlowFileUuid()); @@ -527,14 +528,14 @@ public class TestStandardProcessSession { Assert.fail("Unexpected event type: " + event); } } - + assertEquals(1, joinCount); assertEquals(1, ff1UpdateCount); assertEquals(1, ff2UpdateCount); - + assertEquals(1, joinCount); } - + @Test public void testForkOneToOneReported() throws IOException { final FlowFileRecord flowFileRecord = new StandardFlowFileRecord.Builder() @@ -844,34 +845,34 @@ public class TestStandardProcessSession { @Test public void testContentNotFoundExceptionThrownWhenUnableToReadDataOffsetTooLarge() { final FlowFileRecord flowFileRecord = new StandardFlowFileRecord.Builder() - .addAttribute("uuid", "12345678-1234-1234-1234-123456789012") - .entryDate(System.currentTimeMillis()) - .contentClaim(new ContentClaim() { - @Override - public int compareTo(ContentClaim arg0) { - return 0; - } - - @Override - public String getId() { - return "0"; - } - - @Override - public String getContainer() { - return "container"; - } - - @Override - public String getSection() { - return "section"; - } - - @Override - public boolean isLossTolerant() { - return true; - } - }).build(); + .addAttribute("uuid", "12345678-1234-1234-1234-123456789012") + .entryDate(System.currentTimeMillis()) + .contentClaim(new ContentClaim() { + @Override + public int compareTo(ContentClaim arg0) { + return 0; + } + + @Override + public String getId() { + return "0"; + } + + @Override + public String getContainer() { + return "container"; + } + + @Override + public String getSection() { + return "section"; + } + + @Override + public boolean isLossTolerant() { + return true; + } + }).build(); flowFileQueue.put(flowFileRecord); FlowFile ff1 = session.get(); @@ -884,35 +885,35 @@ public class TestStandardProcessSession { session.commit(); final FlowFileRecord flowFileRecord2 = new StandardFlowFileRecord.Builder() - .addAttribute("uuid", "12345678-1234-1234-1234-123456789012") - .entryDate(System.currentTimeMillis()) - .contentClaim(new ContentClaim() { - @Override - public int compareTo(ContentClaim arg0) { - return 0; - } - - @Override - public String getId() { - return "0"; - } - - @Override - public String getContainer() { - return "container"; - } - - @Override - public String getSection() { - return "section"; - } - - @Override - public boolean isLossTolerant() { - return true; - } - }) - .contentClaimOffset(1000L).size(1L).build(); + .addAttribute("uuid", "12345678-1234-1234-1234-123456789012") + .entryDate(System.currentTimeMillis()) + .contentClaim(new ContentClaim() { + @Override + public int compareTo(ContentClaim arg0) { + return 0; + } + + @Override + public String getId() { + return "0"; + } + + @Override + public String getContainer() { + return "container"; + } + + @Override + public String getSection() { + return "section"; + } + + @Override + public boolean isLossTolerant() { + return true; + } + }) + .contentClaimOffset(1000L).size(1L).build(); flowFileQueue.put(flowFileRecord2); // attempt to read the data. @@ -973,20 +974,21 @@ public class TestStandardProcessSession { } } + @Test public void testCreateEmitted() throws IOException { FlowFile newFlowFile = session.create(); session.transfer(newFlowFile, new Relationship.Builder().name("A").build()); session.commit(); - + final List events = provenanceRepo.getEvents(0L, 10000); assertFalse(events.isEmpty()); assertEquals(1, events.size()); - + final ProvenanceEventRecord event = events.get(0); assertEquals(ProvenanceEventType.CREATE, event.getEventType()); } - + @Test public void testContentModifiedNotEmittedForCreate() throws IOException { FlowFile newFlowFile = session.create(); @@ -997,23 +999,23 @@ public class TestStandardProcessSession { }); session.transfer(newFlowFile, new Relationship.Builder().name("A").build()); session.commit(); - + final List events = provenanceRepo.getEvents(0L, 10000); assertFalse(events.isEmpty()); assertEquals(1, events.size()); - + final ProvenanceEventRecord event = events.get(0); assertEquals(ProvenanceEventType.CREATE, event.getEventType()); } - + @Test public void testContentModifiedEmittedAndNotAttributesModified() throws IOException { final FlowFileRecord flowFile = new StandardFlowFileRecord.Builder() - .id(1L) - .addAttribute("uuid", "000000000000-0000-0000-0000-00000000") - .build(); + .id(1L) + .addAttribute("uuid", "000000000000-0000-0000-0000-00000000") + .build(); this.flowFileQueue.put(flowFile); - + FlowFile existingFlowFile = session.get(); existingFlowFile = session.write(existingFlowFile, new OutputStreamCallback() { @Override @@ -1023,36 +1025,38 @@ public class TestStandardProcessSession { existingFlowFile = session.putAttribute(existingFlowFile, "attr", "a"); session.transfer(existingFlowFile, new Relationship.Builder().name("A").build()); session.commit(); - + final List events = provenanceRepo.getEvents(0L, 10000); assertFalse(events.isEmpty()); assertEquals(1, events.size()); - + final ProvenanceEventRecord event = events.get(0); assertEquals(ProvenanceEventType.CONTENT_MODIFIED, event.getEventType()); } - + @Test public void testAttributesModifiedEmitted() throws IOException { final FlowFileRecord flowFile = new StandardFlowFileRecord.Builder() - .id(1L) - .addAttribute("uuid", "000000000000-0000-0000-0000-00000000") - .build(); + .id(1L) + .addAttribute("uuid", "000000000000-0000-0000-0000-00000000") + .build(); this.flowFileQueue.put(flowFile); - + FlowFile existingFlowFile = session.get(); existingFlowFile = session.putAttribute(existingFlowFile, "attr", "a"); session.transfer(existingFlowFile, new Relationship.Builder().name("A").build()); session.commit(); - + final List events = provenanceRepo.getEvents(0L, 10000); assertFalse(events.isEmpty()); assertEquals(1, events.size()); - + final ProvenanceEventRecord event = events.get(0); assertEquals(ProvenanceEventType.ATTRIBUTES_MODIFIED, event.getEventType()); } - + + + private static class MockFlowFileRepository implements FlowFileRepository { private final AtomicLong idGenerator = new AtomicLong(0L); @@ -1119,7 +1123,7 @@ public class TestStandardProcessSession { @Override public void shutdown() { } - + public Set getExistingClaims() { final Set claims = new HashSet<>(); @@ -1142,7 +1146,7 @@ public class TestStandardProcessSession { if (Files.exists(parent) == false) { Files.createDirectories(parent); } - Files.createFile(getPath(claim)); + Files.createFile(getPath(claim)); return claim; } From ad18853b589d80331e2f4574bce35d79bce09c28 Mon Sep 17 00:00:00 2001 From: joewitt Date: Wed, 18 Mar 2015 10:59:13 -0400 Subject: [PATCH 034/116] NIFI-399 initial port --- .../standard/EvaluateRegularExpression.java | 7 +- .../nifi/processors/standard/ExtractText.java | 294 ++++++++++++++++++ .../org.apache.nifi.processor.Processor | 1 + ...arExpression.java => TestExtractText.java} | 92 +++--- 4 files changed, 345 insertions(+), 49 deletions(-) create mode 100644 nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ExtractText.java rename nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/{TestEvaluateRegularExpression.java => TestExtractText.java} (80%) diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateRegularExpression.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateRegularExpression.java index 4140943547..bb2e31a84c 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateRegularExpression.java +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateRegularExpression.java @@ -52,9 +52,10 @@ import org.apache.commons.lang3.StringUtils; @EventDriven @SideEffectFree @SupportsBatching -@Tags({"evaluate", "Text", "Regular Expression", "regex", "experimental"}) +@Tags({"deprecated"}) @CapabilityDescription( - "Evaluates one or more Regular Expressions against the content of a FlowFile. " + "WARNING: This has been deprecated and will be removed in 0.2.0. Use ExtractText instead.\n" + + "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. " @@ -62,7 +63,7 @@ import org.apache.commons.lang3.StringUtils; + "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.") - +@Deprecated public class EvaluateRegularExpression extends AbstractProcessor { public static final PropertyDescriptor CHARACTER_SET = new PropertyDescriptor.Builder() diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ExtractText.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ExtractText.java new file mode 100644 index 0000000000..6c914d899d --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ExtractText.java @@ -0,0 +1,294 @@ +/* + * 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.standard; + +import java.io.IOException; +import java.io.InputStream; +import java.nio.charset.Charset; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.flowfile.FlowFile; +import org.apache.nifi.stream.io.StreamUtils; +import org.apache.nifi.logging.ProcessorLog; +import org.apache.nifi.processor.AbstractProcessor; +import org.apache.nifi.processor.DataUnit; +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.behavior.EventDriven; +import org.apache.nifi.annotation.behavior.SideEffectFree; +import org.apache.nifi.annotation.behavior.SupportsBatching; +import org.apache.nifi.annotation.documentation.Tags; +import org.apache.nifi.processor.io.InputStreamCallback; +import org.apache.nifi.processor.util.StandardValidators; + +import org.apache.commons.lang3.StringUtils; + +@EventDriven +@SideEffectFree +@SupportsBatching +@Tags({"evaluate", "extract", "Text", "Regular Expression", "regex"}) +@CapabilityDescription( + "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.") + +public class ExtractText extends AbstractProcessor { + + public static final PropertyDescriptor CHARACTER_SET = new PropertyDescriptor.Builder() + .name("Character Set") + .description("The Character Set in which the file is encoded") + .required(true) + .addValidator(StandardValidators.CHARACTER_SET_VALIDATOR) + .defaultValue("UTF-8") + .build(); + + public static final PropertyDescriptor MAX_BUFFER_SIZE = new PropertyDescriptor.Builder() + .name("Maximum Buffer Size") + .description("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.") + .required(true) + .addValidator(StandardValidators.DATA_SIZE_VALIDATOR) + .defaultValue("1 MB") + .build(); + + public static final PropertyDescriptor CANON_EQ = new PropertyDescriptor.Builder() + .name("Enable Canonical Equivalence") + .description("Indicates that two characters match only when their full canonical decompositions match.") + .required(true) + .allowableValues("true", "false") + .defaultValue("false") + .build(); + + public static final PropertyDescriptor CASE_INSENSITIVE = new PropertyDescriptor.Builder() + .name("Enable Case-insensitive Matching") + .description("Indicates that two characters match even if they are in a different case. Can also be specified via the embeded flag (?i).") + .required(true) + .allowableValues("true", "false") + .defaultValue("false") + .build(); + + public static final PropertyDescriptor COMMENTS = new PropertyDescriptor.Builder() + .name("Permit Whitespace and Comments in Pattern") + .description("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).") + .required(true) + .allowableValues("true", "false") + .defaultValue("false") + .build(); + + public static final PropertyDescriptor DOTALL = new PropertyDescriptor.Builder() + .name("Enable DOTALL Mode") + .description("Indicates that the expression '.' should match any character, including a line terminator. Can also be specified via the embeded flag (?s).") + .required(true) + .allowableValues("true", "false") + .defaultValue("false") + .build(); + + public static final PropertyDescriptor LITERAL = new PropertyDescriptor.Builder() + .name("Enable Literal Parsing of the Pattern") + .description("Indicates that Metacharacters and escape characters should be given no special meaning.") + .required(true) + .allowableValues("true", "false") + .defaultValue("false") + .build(); + + public static final PropertyDescriptor MULTILINE = new PropertyDescriptor.Builder() + .name("Enable Multiline Mode") + .description("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).") + .required(true) + .allowableValues("true", "false") + .defaultValue("false") + .build(); + + public static final PropertyDescriptor UNICODE_CASE = new PropertyDescriptor.Builder() + .name("Enable Unicode-aware Case Folding") + .description("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).") + .required(true) + .allowableValues("true", "false") + .defaultValue("false") + .build(); + + public static final PropertyDescriptor UNICODE_CHARACTER_CLASS = new PropertyDescriptor.Builder() + .name("Enable Unicode Predefined Character Classes") + .description("Specifies conformance with the Unicode Technical Standard #18: Unicode Regular Expression Annex C: Compatibility Properties. Can also be specified via the embeded flag (?U).") + .required(true) + .allowableValues("true", "false") + .defaultValue("false") + .build(); + + public static final PropertyDescriptor UNIX_LINES = new PropertyDescriptor.Builder() + .name("Enable Unix Lines Mode") + .description("Indicates that only the '\n' line terminator is recognized int the behavior of '.', '^', and '$'. Can also be specified via the embeded flag (?d).") + .required(true) + .allowableValues("true", "false") + .defaultValue("false") + .build(); + + public static final Relationship REL_MATCH = new Relationship.Builder() + .name("matched") + .description( + "FlowFiles are routed to this relationship when the Regular Expression is successfully evaluated and the FlowFile " + + "is modified as a result") + .build(); + + public static final Relationship REL_NO_MATCH = new Relationship.Builder() + .name("unmatched") + .description( + "FlowFiles are routed to this relationship when no provided Regular Expression matches the content of the FlowFile") + .build(); + + private Set relationships; + private List properties; + + @Override + protected void init(final ProcessorInitializationContext context) { + final Set relationships = new HashSet<>(); + relationships.add(REL_MATCH); + relationships.add(REL_NO_MATCH); + this.relationships = Collections.unmodifiableSet(relationships); + + final List properties = new ArrayList<>(); + properties.add(CHARACTER_SET); + properties.add(MAX_BUFFER_SIZE); + properties.add(CANON_EQ); + properties.add(CASE_INSENSITIVE); + properties.add(COMMENTS); + properties.add(DOTALL); + properties.add(LITERAL); + properties.add(MULTILINE); + properties.add(UNICODE_CASE); + properties.add(UNICODE_CHARACTER_CLASS); + properties.add(UNIX_LINES); + this.properties = Collections.unmodifiableList(properties); + } + + @Override + public Set getRelationships() { + return relationships; + } + + @Override + protected List getSupportedPropertyDescriptors() { + return properties; + } + + @Override + protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) { + return new PropertyDescriptor.Builder() + .name(propertyDescriptorName) + .expressionLanguageSupported(false) + .addValidator(StandardValidators.createRegexValidator(1, 1, true)) + .required(false) + .dynamic(true) + .build(); + } + + @Override + public void onTrigger(final ProcessContext context, final ProcessSession session) { + final List flowFileBatch = session.get(50); + if (flowFileBatch.isEmpty()) { + return; + } + final ProcessorLog logger = getLogger(); + + // Compile the Regular Expressions + Map regexMap = new HashMap<>(); + for (final Map.Entry entry : context.getProperties().entrySet()) { + if (!entry.getKey().isDynamic()) { + continue; + } + final int flags = getCompileFlags(context); + final Matcher matcher = Pattern.compile(entry.getValue(), flags).matcher(""); + regexMap.put(entry.getKey().getName(), matcher); + } + + final Charset charset = Charset.forName(context.getProperty(CHARACTER_SET).getValue()); + + final int maxBufferSize = context.getProperty(MAX_BUFFER_SIZE).asDataSize(DataUnit.B).intValue(); + + for (FlowFile flowFile : flowFileBatch) { + + final Map regexResults = new HashMap<>(); + + final byte[] buffer = new byte[maxBufferSize]; + + session.read(flowFile, new InputStreamCallback() { + @Override + public void process(InputStream in) throws IOException { + StreamUtils.fillBuffer(in, buffer, false); + } + }); + + final int flowFileSize = Math.min((int) flowFile.getSize(), maxBufferSize); + + final String contentString = new String(buffer, 0, flowFileSize, charset); + + for (final Map.Entry entry : regexMap.entrySet()) { + + final Matcher matcher = entry.getValue(); + + matcher.reset(contentString); + + if (matcher.find()) { + final String group = matcher.group(1); + if (!StringUtils.isBlank(group)) { + regexResults.put(entry.getKey(), group); + } + } + } + + if (!regexResults.isEmpty()) { + flowFile = session.putAllAttributes(flowFile, regexResults); + session.getProvenanceReporter().modifyAttributes(flowFile); + session.transfer(flowFile, REL_MATCH); + logger.info("Matched {} Regular Expressions and added attributes to FlowFile {}", new Object[]{regexResults.size(), flowFile}); + } else { + session.transfer(flowFile, REL_NO_MATCH); + logger.info("Did not match any Regular Expressions for FlowFile {}", new Object[]{flowFile}); + } + + } // end flowFileLoop + } + + int getCompileFlags(ProcessContext context) { + int flags = (context.getProperty(UNIX_LINES).asBoolean() ? Pattern.UNIX_LINES : 0) + | (context.getProperty(CASE_INSENSITIVE).asBoolean() ? Pattern.CASE_INSENSITIVE : 0) + | (context.getProperty(COMMENTS).asBoolean() ? Pattern.COMMENTS : 0) + | (context.getProperty(MULTILINE).asBoolean() ? Pattern.MULTILINE : 0) + | (context.getProperty(LITERAL).asBoolean() ? Pattern.LITERAL : 0) + | (context.getProperty(DOTALL).asBoolean() ? Pattern.DOTALL : 0) + | (context.getProperty(UNICODE_CASE).asBoolean() ? Pattern.UNICODE_CASE : 0) + | (context.getProperty(CANON_EQ).asBoolean() ? Pattern.CANON_EQ : 0) + | (context.getProperty(UNICODE_CHARACTER_CLASS).asBoolean() ? Pattern.UNICODE_CHARACTER_CLASS : 0); + return flags; + } +} diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor index f81ccec9c4..7fbd7817a6 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor @@ -25,6 +25,7 @@ org.apache.nifi.processors.standard.EvaluateXPath org.apache.nifi.processors.standard.EvaluateXQuery org.apache.nifi.processors.standard.ExecuteStreamCommand org.apache.nifi.processors.standard.ExecuteProcess +org.apache.nifi.processors.standard.ExtractText org.apache.nifi.processors.standard.GenerateFlowFile org.apache.nifi.processors.standard.GetFile org.apache.nifi.processors.standard.GetFTP diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestEvaluateRegularExpression.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestExtractText.java similarity index 80% rename from nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestEvaluateRegularExpression.java rename to nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestExtractText.java index c1e5b3ceb5..355d2552f7 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestEvaluateRegularExpression.java +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestExtractText.java @@ -16,7 +16,7 @@ */ package org.apache.nifi.processors.standard; -import org.apache.nifi.processors.standard.EvaluateRegularExpression; +import org.apache.nifi.processors.standard.ExtractText; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; @@ -31,14 +31,14 @@ import org.apache.nifi.util.TestRunners; import org.junit.Test; -public class TestEvaluateRegularExpression { +public class TestExtractText { final String SAMPLE_STRING = "foo\r\nbar1\r\nbar2\r\nbar3\r\nhello\r\nworld\r\n"; @Test public void testProcessor() throws Exception { - final TestRunner testRunner = TestRunners.newTestRunner(new EvaluateRegularExpression()); + final TestRunner testRunner = TestRunners.newTestRunner(new ExtractText()); testRunner.setProperty("regex.result1", "(?s)(.*)"); testRunner.setProperty("regex.result2", "(?s).*(bar1).*"); @@ -51,8 +51,8 @@ public class TestEvaluateRegularExpression { testRunner.enqueue(SAMPLE_STRING.getBytes("UTF-8")); testRunner.run(); - testRunner.assertAllFlowFilesTransferred(EvaluateRegularExpression.REL_MATCH, 1); - final MockFlowFile out = testRunner.getFlowFilesForRelationship(EvaluateRegularExpression.REL_MATCH).get(0); + testRunner.assertAllFlowFilesTransferred(ExtractText.REL_MATCH, 1); + final MockFlowFile out = testRunner.getFlowFilesForRelationship(ExtractText.REL_MATCH).get(0); out.assertAttributeEquals("regex.result1", SAMPLE_STRING); out.assertAttributeEquals("regex.result2", "bar1"); out.assertAttributeEquals("regex.result3", "bar1"); @@ -65,9 +65,9 @@ public class TestEvaluateRegularExpression { @Test public void testProcessorWithDotall() throws Exception { - final TestRunner testRunner = TestRunners.newTestRunner(new EvaluateRegularExpression()); + final TestRunner testRunner = TestRunners.newTestRunner(new ExtractText()); - testRunner.setProperty(EvaluateRegularExpression.DOTALL, "true"); + testRunner.setProperty(ExtractText.DOTALL, "true"); testRunner.setProperty("regex.result1", "(.*)"); testRunner.setProperty("regex.result2", ".*(bar1).*"); @@ -80,8 +80,8 @@ public class TestEvaluateRegularExpression { testRunner.enqueue(SAMPLE_STRING.getBytes("UTF-8")); testRunner.run(); - testRunner.assertAllFlowFilesTransferred(EvaluateRegularExpression.REL_MATCH, 1); - final MockFlowFile out = testRunner.getFlowFilesForRelationship(EvaluateRegularExpression.REL_MATCH).get(0); + testRunner.assertAllFlowFilesTransferred(ExtractText.REL_MATCH, 1); + final MockFlowFile out = testRunner.getFlowFilesForRelationship(ExtractText.REL_MATCH).get(0); out.assertAttributeEquals("regex.result1", SAMPLE_STRING); out.assertAttributeEquals("regex.result2", "bar1"); out.assertAttributeEquals("regex.result3", "bar1"); @@ -95,9 +95,9 @@ public class TestEvaluateRegularExpression { @Test public void testProcessorWithMultiline() throws Exception { - final TestRunner testRunner = TestRunners.newTestRunner(new EvaluateRegularExpression()); + final TestRunner testRunner = TestRunners.newTestRunner(new ExtractText()); - testRunner.setProperty(EvaluateRegularExpression.MULTILINE, "true"); + testRunner.setProperty(ExtractText.MULTILINE, "true"); testRunner.setProperty("regex.result1", "(.*)"); testRunner.setProperty("regex.result2", "(bar1)"); @@ -112,8 +112,8 @@ public class TestEvaluateRegularExpression { testRunner.enqueue(SAMPLE_STRING.getBytes("UTF-8")); testRunner.run(); - testRunner.assertAllFlowFilesTransferred(EvaluateRegularExpression.REL_MATCH, 1); - final MockFlowFile out = testRunner.getFlowFilesForRelationship(EvaluateRegularExpression.REL_MATCH).get(0); + testRunner.assertAllFlowFilesTransferred(ExtractText.REL_MATCH, 1); + final MockFlowFile out = testRunner.getFlowFilesForRelationship(ExtractText.REL_MATCH).get(0); out.assertAttributeEquals("regex.result1", "foo"); // matches everything on the first line out.assertAttributeEquals("regex.result2", "bar1"); out.assertAttributeEquals("regex.result3", "bar1"); @@ -128,10 +128,10 @@ public class TestEvaluateRegularExpression { @Test public void testProcessorWithMultilineAndDotall() throws Exception { - final TestRunner testRunner = TestRunners.newTestRunner(new EvaluateRegularExpression()); + final TestRunner testRunner = TestRunners.newTestRunner(new ExtractText()); - testRunner.setProperty(EvaluateRegularExpression.MULTILINE, "true"); - testRunner.setProperty(EvaluateRegularExpression.DOTALL, "true"); + testRunner.setProperty(ExtractText.MULTILINE, "true"); + testRunner.setProperty(ExtractText.DOTALL, "true"); testRunner.setProperty("regex.result1", "(.*)"); testRunner.setProperty("regex.result2", "(bar1)"); @@ -146,8 +146,8 @@ public class TestEvaluateRegularExpression { testRunner.enqueue(SAMPLE_STRING.getBytes("UTF-8")); testRunner.run(); - testRunner.assertAllFlowFilesTransferred(EvaluateRegularExpression.REL_MATCH, 1); - final MockFlowFile out = testRunner.getFlowFilesForRelationship(EvaluateRegularExpression.REL_MATCH).get(0); + testRunner.assertAllFlowFilesTransferred(ExtractText.REL_MATCH, 1); + final MockFlowFile out = testRunner.getFlowFilesForRelationship(ExtractText.REL_MATCH).get(0); out.assertAttributeEquals("regex.result1", SAMPLE_STRING); out.assertAttributeEquals("regex.result2", "bar1"); @@ -163,10 +163,10 @@ public class TestEvaluateRegularExpression { @Test public void testProcessorWithNoMatches() throws Exception { - final TestRunner testRunner = TestRunners.newTestRunner(new EvaluateRegularExpression()); + final TestRunner testRunner = TestRunners.newTestRunner(new ExtractText()); - testRunner.setProperty(EvaluateRegularExpression.MULTILINE, "true"); - testRunner.setProperty(EvaluateRegularExpression.DOTALL, "true"); + testRunner.setProperty(ExtractText.MULTILINE, "true"); + testRunner.setProperty(ExtractText.DOTALL, "true"); testRunner.setProperty("regex.result2", "(bar1)"); testRunner.setProperty("regex.result3", ".*?(bar\\d).*"); @@ -179,8 +179,8 @@ public class TestEvaluateRegularExpression { testRunner.enqueue("YYY".getBytes("UTF-8")); testRunner.run(); - testRunner.assertAllFlowFilesTransferred(EvaluateRegularExpression.REL_NO_MATCH, 1); - final MockFlowFile out = testRunner.getFlowFilesForRelationship(EvaluateRegularExpression.REL_NO_MATCH).get(0); + testRunner.assertAllFlowFilesTransferred(ExtractText.REL_NO_MATCH, 1); + final MockFlowFile out = testRunner.getFlowFilesForRelationship(ExtractText.REL_NO_MATCH).get(0); out.assertAttributeEquals("regex.result1", null); out.assertAttributeEquals("regex.result2", null); @@ -195,7 +195,7 @@ public class TestEvaluateRegularExpression { @Test(expected = java.lang.AssertionError.class) public void testNoCaptureGroups() throws UnsupportedEncodingException { - final TestRunner testRunner = TestRunners.newTestRunner(new EvaluateRegularExpression()); + final TestRunner testRunner = TestRunners.newTestRunner(new ExtractText()); testRunner.setProperty("regex.result1", ".*"); testRunner.enqueue(SAMPLE_STRING.getBytes("UTF-8")); testRunner.run(); @@ -203,15 +203,15 @@ public class TestEvaluateRegularExpression { @Test public void testNoFlowFile() throws UnsupportedEncodingException { - final TestRunner testRunner = TestRunners.newTestRunner(new EvaluateRegularExpression()); + final TestRunner testRunner = TestRunners.newTestRunner(new ExtractText()); testRunner.run(); - testRunner.assertAllFlowFilesTransferred(EvaluateRegularExpression.REL_MATCH, 0); + testRunner.assertAllFlowFilesTransferred(ExtractText.REL_MATCH, 0); } @Test(expected = java.lang.AssertionError.class) public void testTooManyCaptureGroups() throws UnsupportedEncodingException { - final TestRunner testRunner = TestRunners.newTestRunner(new EvaluateRegularExpression()); + final TestRunner testRunner = TestRunners.newTestRunner(new ExtractText()); testRunner.setProperty("regex.result1", "(.)(.)"); testRunner.enqueue(SAMPLE_STRING.getBytes("UTF-8")); testRunner.run(); @@ -219,9 +219,9 @@ public class TestEvaluateRegularExpression { @Test public void testMatchOutsideBuffer() throws Exception { - final TestRunner testRunner = TestRunners.newTestRunner(new EvaluateRegularExpression()); + final TestRunner testRunner = TestRunners.newTestRunner(new ExtractText()); - testRunner.setProperty(EvaluateRegularExpression.MAX_BUFFER_SIZE, "3 B");//only read the first 3 chars ("foo") + testRunner.setProperty(ExtractText.MAX_BUFFER_SIZE, "3 B");//only read the first 3 chars ("foo") testRunner.setProperty("regex.result1", "(foo)"); testRunner.setProperty("regex.result2", "(world)"); @@ -229,8 +229,8 @@ public class TestEvaluateRegularExpression { testRunner.enqueue(SAMPLE_STRING.getBytes("UTF-8")); testRunner.run(); - testRunner.assertAllFlowFilesTransferred(EvaluateRegularExpression.REL_MATCH, 1); - final MockFlowFile out = testRunner.getFlowFilesForRelationship(EvaluateRegularExpression.REL_MATCH).get(0); + testRunner.assertAllFlowFilesTransferred(ExtractText.REL_MATCH, 1); + final MockFlowFile out = testRunner.getFlowFilesForRelationship(ExtractText.REL_MATCH).get(0); out.assertAttributeEquals("regex.result1", "foo"); out.assertAttributeEquals("regex.result2", null); // null because outsk @@ -239,7 +239,7 @@ public class TestEvaluateRegularExpression { @Test public void testGetCompileFlags() { - final EvaluateRegularExpression processor = new EvaluateRegularExpression(); + final ExtractText processor = new ExtractText(); TestRunner testRunner; int flags; @@ -250,60 +250,60 @@ public class TestEvaluateRegularExpression { // UNIX_LINES testRunner = TestRunners.newTestRunner(processor); - testRunner.setProperty(EvaluateRegularExpression.UNIX_LINES, "true"); + testRunner.setProperty(ExtractText.UNIX_LINES, "true"); assertEquals(Pattern.UNIX_LINES, processor.getCompileFlags(testRunner.getProcessContext())); // CASE_INSENSITIVE testRunner = TestRunners.newTestRunner(processor); - testRunner.setProperty(EvaluateRegularExpression.CASE_INSENSITIVE, "true"); + testRunner.setProperty(ExtractText.CASE_INSENSITIVE, "true"); assertEquals(Pattern.CASE_INSENSITIVE, processor.getCompileFlags(testRunner.getProcessContext())); // COMMENTS testRunner = TestRunners.newTestRunner(processor); - testRunner.setProperty(EvaluateRegularExpression.COMMENTS, "true"); + testRunner.setProperty(ExtractText.COMMENTS, "true"); assertEquals(Pattern.COMMENTS, processor.getCompileFlags(testRunner.getProcessContext())); // MULTILINE testRunner = TestRunners.newTestRunner(processor); - testRunner.setProperty(EvaluateRegularExpression.MULTILINE, "true"); + testRunner.setProperty(ExtractText.MULTILINE, "true"); assertEquals(Pattern.MULTILINE, processor.getCompileFlags(testRunner.getProcessContext())); // LITERAL testRunner = TestRunners.newTestRunner(processor); - testRunner.setProperty(EvaluateRegularExpression.LITERAL, "true"); + testRunner.setProperty(ExtractText.LITERAL, "true"); assertEquals(Pattern.LITERAL, processor.getCompileFlags(testRunner.getProcessContext())); // DOTALL testRunner = TestRunners.newTestRunner(processor); - testRunner.setProperty(EvaluateRegularExpression.DOTALL, "true"); + testRunner.setProperty(ExtractText.DOTALL, "true"); assertEquals(Pattern.DOTALL, processor.getCompileFlags(testRunner.getProcessContext())); // UNICODE_CASE testRunner = TestRunners.newTestRunner(processor); - testRunner.setProperty(EvaluateRegularExpression.UNICODE_CASE, "true"); + testRunner.setProperty(ExtractText.UNICODE_CASE, "true"); assertEquals(Pattern.UNICODE_CASE, processor.getCompileFlags(testRunner.getProcessContext())); // CANON_EQ testRunner = TestRunners.newTestRunner(processor); - testRunner.setProperty(EvaluateRegularExpression.CANON_EQ, "true"); + testRunner.setProperty(ExtractText.CANON_EQ, "true"); assertEquals(Pattern.CANON_EQ, processor.getCompileFlags(testRunner.getProcessContext())); // UNICODE_CHARACTER_CLASS testRunner = TestRunners.newTestRunner(processor); - testRunner.setProperty(EvaluateRegularExpression.UNICODE_CHARACTER_CLASS, "true"); + testRunner.setProperty(ExtractText.UNICODE_CHARACTER_CLASS, "true"); assertEquals(Pattern.UNICODE_CHARACTER_CLASS, processor.getCompileFlags(testRunner.getProcessContext())); // DOTALL and MULTILINE testRunner = TestRunners.newTestRunner(processor); - testRunner.setProperty(EvaluateRegularExpression.DOTALL, "true"); - testRunner.setProperty(EvaluateRegularExpression.MULTILINE, "true"); + testRunner.setProperty(ExtractText.DOTALL, "true"); + testRunner.setProperty(ExtractText.MULTILINE, "true"); assertEquals(Pattern.DOTALL | Pattern.MULTILINE, processor.getCompileFlags(testRunner.getProcessContext())); } @Test public void testGetRelationShips() throws Exception { - final EvaluateRegularExpression processor = new EvaluateRegularExpression(); + final ExtractText processor = new ExtractText(); final TestRunner testRunner = TestRunners.newTestRunner(processor); // testRunner.setProperty("regex.result1", "(.*)"); @@ -311,8 +311,8 @@ public class TestEvaluateRegularExpression { testRunner.run(); Set relationships = processor.getRelationships(); - assertTrue(relationships.contains(EvaluateRegularExpression.REL_MATCH)); - assertTrue(relationships.contains(EvaluateRegularExpression.REL_NO_MATCH)); + assertTrue(relationships.contains(ExtractText.REL_MATCH)); + assertTrue(relationships.contains(ExtractText.REL_NO_MATCH)); assertEquals(2, relationships.size()); } From a6740a6e2c87f4c994d305db55c0777dc4f99976 Mon Sep 17 00:00:00 2001 From: joewitt Date: Thu, 19 Mar 2015 01:21:32 -0400 Subject: [PATCH 035/116] NIFI-399 addressed items in the ticket --- .../processor/util/StandardValidators.java | 22 +++ .../util/TestStandardValidators.java | 31 ++++ .../nifi/processors/standard/ExtractText.java | 153 ++++++++++-------- .../processors/standard/TestExtractText.java | 15 +- 4 files changed, 144 insertions(+), 77 deletions(-) diff --git a/nifi/nifi-commons/nifi-processor-utilities/src/main/java/org/apache/nifi/processor/util/StandardValidators.java b/nifi/nifi-commons/nifi-processor-utilities/src/main/java/org/apache/nifi/processor/util/StandardValidators.java index 10748fe0af..c9ae60974a 100644 --- a/nifi/nifi-commons/nifi-processor-utilities/src/main/java/org/apache/nifi/processor/util/StandardValidators.java +++ b/nifi/nifi-commons/nifi-processor-utilities/src/main/java/org/apache/nifi/processor/util/StandardValidators.java @@ -287,6 +287,28 @@ public class StandardValidators { return createAttributeExpressionLanguageValidator(expectedResultType, true); } + public static Validator createDataSizeBoundsValidator(final long minBytesInclusive, final long maxBytesInclusive) { + return new Validator() { + + @Override + public ValidationResult validate(final String subject, final String input, final ValidationContext context) { + final ValidationResult vr = DATA_SIZE_VALIDATOR.validate(subject, input, context); + if(!vr.isValid()){ + return vr; + } + final long dataSizeBytes = DataUnit.parseDataSize(input, DataUnit.B).longValue(); + if(dataSizeBytes < minBytesInclusive){ + return new ValidationResult.Builder().subject(subject).input(input).valid(false).explanation("Cannot be smaller than " + minBytesInclusive + " bytes").build(); + } + if(dataSizeBytes > maxBytesInclusive){ + return new ValidationResult.Builder().subject(subject).input(input).valid(false).explanation("Cannot be larger than " + maxBytesInclusive + " bytes").build(); + } + return new ValidationResult.Builder().subject(subject).input(input).valid(true).build(); + } + }; + + } + public static Validator createRegexMatchingValidator(final Pattern pattern) { return new Validator() { @Override diff --git a/nifi/nifi-commons/nifi-processor-utilities/src/test/java/org/apache/nifi/processor/util/TestStandardValidators.java b/nifi/nifi-commons/nifi-processor-utilities/src/test/java/org/apache/nifi/processor/util/TestStandardValidators.java index 2ae50c91c7..70b8d215ac 100644 --- a/nifi/nifi-commons/nifi-processor-utilities/src/test/java/org/apache/nifi/processor/util/TestStandardValidators.java +++ b/nifi/nifi-commons/nifi-processor-utilities/src/test/java/org/apache/nifi/processor/util/TestStandardValidators.java @@ -51,4 +51,35 @@ public class TestStandardValidators { vr = val.validate("TimePeriodTest", "1 sec", null); assertTrue(vr.isValid()); } + + @Test + public void testDataSizeBoundsValidator() { + Validator val = StandardValidators.createDataSizeBoundsValidator(100, 1000); + ValidationResult vr; + + vr = val.validate("DataSizeBounds", "5 GB", null); + assertFalse(vr.isValid()); + + vr = val.validate("DataSizeBounds", "0 B", null); + assertFalse(vr.isValid()); + + vr = val.validate("DataSizeBounds", "99 B", null); + assertFalse(vr.isValid()); + + vr = val.validate("DataSizeBounds", "100 B", null); + assertTrue(vr.isValid()); + + vr = val.validate("DataSizeBounds", "999 B", null); + assertTrue(vr.isValid()); + + vr = val.validate("DataSizeBounds", "1000 B", null); + assertTrue(vr.isValid()); + + vr = val.validate("DataSizeBounds", "1001 B", null); + assertFalse(vr.isValid()); + + vr = val.validate("DataSizeBounds", "water", null); + assertFalse(vr.isValid()); + + } } diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ExtractText.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ExtractText.java index 6c914d899d..aa1b7a95b0 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ExtractText.java +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ExtractText.java @@ -26,6 +26,7 @@ import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Set; +import java.util.concurrent.atomic.AtomicReference; import java.util.regex.Matcher; import java.util.regex.Pattern; @@ -47,7 +48,7 @@ import org.apache.nifi.annotation.documentation.Tags; import org.apache.nifi.processor.io.InputStreamCallback; import org.apache.nifi.processor.util.StandardValidators; -import org.apache.commons.lang3.StringUtils; +import org.apache.nifi.annotation.lifecycle.OnScheduled; @EventDriven @SideEffectFree @@ -58,10 +59,14 @@ import org.apache.commons.lang3.StringUtils; + "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. " + + "The first capture group, if any found, will be placed into that attribute name." + + "But all catpure groups, including the matching string sequence itself will also be " + + "provided at that attribute name with an index value provided." + + "The value of the property must be a valid Regular Expressions with one or more capturing groups. " + "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.") + + "If no provided Regular Expression matches, the FlowFile will be routed to 'unmatched' " + + "and no attributes will be applied to the FlowFile.") public class ExtractText extends AbstractProcessor { @@ -78,9 +83,18 @@ public class ExtractText extends AbstractProcessor { .description("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.") .required(true) .addValidator(StandardValidators.DATA_SIZE_VALIDATOR) + .addValidator(StandardValidators.createDataSizeBoundsValidator(0, Integer.MAX_VALUE)) .defaultValue("1 MB") .build(); + public static final PropertyDescriptor MAX_CAPTURE_GROUP_LENGTH = new PropertyDescriptor.Builder() + .name("Maximum Capture Group Length") + .description("Specifies the maximum number of characters a given capture group value can have. Any characters beyond the max will be truncated.") + .required(false) + .defaultValue("1024") + .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR) + .build(); + public static final PropertyDescriptor CANON_EQ = new PropertyDescriptor.Builder() .name("Enable Canonical Equivalence") .description("Indicates that two characters match only when their full canonical decompositions match.") @@ -168,27 +182,29 @@ public class ExtractText extends AbstractProcessor { private Set relationships; private List properties; + private final AtomicReference> compiledPattersMapRef = new AtomicReference<>(); @Override protected void init(final ProcessorInitializationContext context) { - final Set relationships = new HashSet<>(); - relationships.add(REL_MATCH); - relationships.add(REL_NO_MATCH); - this.relationships = Collections.unmodifiableSet(relationships); + final Set rels = new HashSet<>(); + rels.add(REL_MATCH); + rels.add(REL_NO_MATCH); + this.relationships = Collections.unmodifiableSet(rels); - final List properties = new ArrayList<>(); - properties.add(CHARACTER_SET); - properties.add(MAX_BUFFER_SIZE); - properties.add(CANON_EQ); - properties.add(CASE_INSENSITIVE); - properties.add(COMMENTS); - properties.add(DOTALL); - properties.add(LITERAL); - properties.add(MULTILINE); - properties.add(UNICODE_CASE); - properties.add(UNICODE_CHARACTER_CLASS); - properties.add(UNIX_LINES); - this.properties = Collections.unmodifiableList(properties); + final List props = new ArrayList<>(); + props.add(CHARACTER_SET); + props.add(MAX_BUFFER_SIZE); + props.add(MAX_CAPTURE_GROUP_LENGTH); + props.add(CANON_EQ); + props.add(CASE_INSENSITIVE); + props.add(COMMENTS); + props.add(DOTALL); + props.add(LITERAL); + props.add(MULTILINE); + props.add(UNICODE_CASE); + props.add(UNICODE_CHARACTER_CLASS); + props.add(UNIX_LINES); + this.properties = Collections.unmodifiableList(props); } @Override @@ -206,77 +222,80 @@ public class ExtractText extends AbstractProcessor { return new PropertyDescriptor.Builder() .name(propertyDescriptorName) .expressionLanguageSupported(false) - .addValidator(StandardValidators.createRegexValidator(1, 1, true)) + .addValidator(StandardValidators.createRegexValidator(1, 40, true)) .required(false) .dynamic(true) .build(); } - @Override - public void onTrigger(final ProcessContext context, final ProcessSession session) { - final List flowFileBatch = session.get(50); - if (flowFileBatch.isEmpty()) { - return; - } - final ProcessorLog logger = getLogger(); + @OnScheduled + public final void onScheduled(final ProcessContext context) throws IOException { + final Map compiledPatternsMap = new HashMap<>(); - // Compile the Regular Expressions - Map regexMap = new HashMap<>(); for (final Map.Entry entry : context.getProperties().entrySet()) { if (!entry.getKey().isDynamic()) { continue; } final int flags = getCompileFlags(context); - final Matcher matcher = Pattern.compile(entry.getValue(), flags).matcher(""); - regexMap.put(entry.getKey().getName(), matcher); + final Pattern pattern = Pattern.compile(entry.getValue(), flags); + compiledPatternsMap.put(entry.getKey().getName(), pattern); } + compiledPattersMapRef.set(compiledPatternsMap); + } + @Override + public void onTrigger(final ProcessContext context, final ProcessSession session) { + FlowFile flowFile = session.get(); + if (flowFile == null) { + return; + } + final ProcessorLog logger = getLogger(); final Charset charset = Charset.forName(context.getProperty(CHARACTER_SET).getValue()); + final int maxCaptureGroupLength = context.getProperty(MAX_CAPTURE_GROUP_LENGTH).asInteger(); + final long maxBufferSizeL = Math.min(flowFile.getSize(), context.getProperty(MAX_BUFFER_SIZE).asDataSize(DataUnit.B).longValue()); + final byte[] buffer = new byte[(int) maxBufferSizeL]; - final int maxBufferSize = context.getProperty(MAX_BUFFER_SIZE).asDataSize(DataUnit.B).intValue(); + session.read(flowFile, new InputStreamCallback() { + @Override + public void process(InputStream in) throws IOException { + StreamUtils.fillBuffer(in, buffer, false); + } + }); - for (FlowFile flowFile : flowFileBatch) { + final String contentString = new String(buffer, 0, (int) maxBufferSizeL, charset); + final Map regexResults = new HashMap<>(); - final Map regexResults = new HashMap<>(); + final Map patternMap = compiledPattersMapRef.get(); + for (final Map.Entry entry : patternMap.entrySet()) { - final byte[] buffer = new byte[maxBufferSize]; + final Matcher matcher = entry.getValue().matcher(contentString); - session.read(flowFile, new InputStreamCallback() { - @Override - public void process(InputStream in) throws IOException { - StreamUtils.fillBuffer(in, buffer, false); - } - }); - - final int flowFileSize = Math.min((int) flowFile.getSize(), maxBufferSize); - - final String contentString = new String(buffer, 0, flowFileSize, charset); - - for (final Map.Entry entry : regexMap.entrySet()) { - - final Matcher matcher = entry.getValue(); - - matcher.reset(contentString); - - if (matcher.find()) { - final String group = matcher.group(1); - if (!StringUtils.isBlank(group)) { - regexResults.put(entry.getKey(), group); + if (matcher.find()) { + final String baseKey = entry.getKey(); + for (int i = 0; i <= matcher.groupCount(); i++) { + final String key = new StringBuilder(baseKey).append(".").append(i).toString(); + String value = matcher.group(i); + if (value.length() > maxCaptureGroupLength) { + value = value.substring(0, maxCaptureGroupLength); + } + regexResults.put(key, value); + if (i == 1) { + regexResults.put(baseKey, value); } } } + } - if (!regexResults.isEmpty()) { - flowFile = session.putAllAttributes(flowFile, regexResults); - session.getProvenanceReporter().modifyAttributes(flowFile); - session.transfer(flowFile, REL_MATCH); - logger.info("Matched {} Regular Expressions and added attributes to FlowFile {}", new Object[]{regexResults.size(), flowFile}); - } else { - session.transfer(flowFile, REL_NO_MATCH); - logger.info("Did not match any Regular Expressions for FlowFile {}", new Object[]{flowFile}); - } + if (!regexResults.isEmpty()) { + flowFile = session.putAllAttributes(flowFile, regexResults); + session.getProvenanceReporter().modifyAttributes(flowFile); + session.transfer(flowFile, REL_MATCH); + logger.info("Matched {} Regular Expressions and added attributes to FlowFile {}", new Object[]{regexResults.size(), flowFile}); + } else { + session.transfer(flowFile, REL_NO_MATCH); + logger.info("Did not match any Regular Expressions for FlowFile {}", new Object[]{flowFile}); + } - } // end flowFileLoop } int getCompileFlags(ProcessContext context) { diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestExtractText.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestExtractText.java index 355d2552f7..202576779b 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestExtractText.java +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestExtractText.java @@ -16,7 +16,6 @@ */ package org.apache.nifi.processors.standard; -import org.apache.nifi.processors.standard.ExtractText; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; @@ -43,7 +42,7 @@ public class TestExtractText { testRunner.setProperty("regex.result1", "(?s)(.*)"); testRunner.setProperty("regex.result2", "(?s).*(bar1).*"); testRunner.setProperty("regex.result3", "(?s).*?(bar\\d).*"); // reluctant gets first - testRunner.setProperty("regex.result4", "(?s).*?(?:bar\\d).*?(bar\\d).*"); // reluctant w/ repeated pattern gets second + testRunner.setProperty("regex.result4", "(?s).*?(?:bar\\d).*?(bar\\d).*?(bar3).*"); // reluctant w/ repeated pattern gets second testRunner.setProperty("regex.result5", "(?s).*(bar\\d).*"); // greedy gets last testRunner.setProperty("regex.result6", "(?s)^(.*)$"); testRunner.setProperty("regex.result7", "(?s)(XXX)"); @@ -57,6 +56,10 @@ public class TestExtractText { out.assertAttributeEquals("regex.result2", "bar1"); out.assertAttributeEquals("regex.result3", "bar1"); out.assertAttributeEquals("regex.result4", "bar2"); + out.assertAttributeEquals("regex.result4.0", SAMPLE_STRING); + out.assertAttributeEquals("regex.result4.1", "bar2"); + out.assertAttributeEquals("regex.result4.2", "bar3"); + out.assertAttributeNotExists("regex.result4.3"); out.assertAttributeEquals("regex.result5", "bar3"); out.assertAttributeEquals("regex.result6", SAMPLE_STRING); out.assertAttributeEquals("regex.result7", null); @@ -209,14 +212,6 @@ public class TestExtractText { } - @Test(expected = java.lang.AssertionError.class) - public void testTooManyCaptureGroups() throws UnsupportedEncodingException { - final TestRunner testRunner = TestRunners.newTestRunner(new ExtractText()); - testRunner.setProperty("regex.result1", "(.)(.)"); - testRunner.enqueue(SAMPLE_STRING.getBytes("UTF-8")); - testRunner.run(); - } - @Test public void testMatchOutsideBuffer() throws Exception { final TestRunner testRunner = TestRunners.newTestRunner(new ExtractText()); From 8b911c5aab2a4b8283510a3423e3c8962a533b96 Mon Sep 17 00:00:00 2001 From: Mark Payne Date: Thu, 19 Mar 2015 11:04:46 -0400 Subject: [PATCH 036/116] NIFI-443: Always start funnels when added to process group, even when autoResumeState is false --- .../org/apache/nifi/groups/ProcessGroup.java | 38 +++++----- .../nifi/controller/FlowController.java | 73 ++++++++++++------- .../nifi/controller/StandardFlowService.java | 21 ++---- .../controller/StandardFlowSynchronizer.java | 6 +- .../nifi/groups/StandardProcessGroup.java | 55 +++----------- .../ApplicationStartupContextListener.java | 6 +- 6 files changed, 90 insertions(+), 109 deletions(-) diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/groups/ProcessGroup.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/groups/ProcessGroup.java index 61be59c349..53b26e1787 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/groups/ProcessGroup.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/groups/ProcessGroup.java @@ -118,7 +118,7 @@ public interface ProcessGroup { void stopProcessing(); /** - * Starts the given Processor + * Enables the given Processor * * @param processor the processor to start * @throws IllegalStateException if the processor is not valid, or is @@ -127,25 +127,19 @@ public interface ProcessGroup { void enableProcessor(ProcessorNode processor); /** - * Starts the given Input Port + * Enables the given Input Port * * @param port */ void enableInputPort(Port port); /** - * Starts the given Output Port + * Enables the given Output Port * * @param port */ void enableOutputPort(Port port); - /** - * Starts the given Funnel - * - * @param funnel - */ - void enableFunnel(Funnel funnel); /** * Starts the given Processor @@ -206,7 +200,7 @@ public interface ProcessGroup { void stopFunnel(Funnel funnel); /** - * Starts the given Processor + * Disables the given Processor * * @param processor the processor to start * @throws IllegalStateException if the processor is not valid, or is @@ -215,25 +209,19 @@ public interface ProcessGroup { void disableProcessor(ProcessorNode processor); /** - * Starts the given Input Port + * Disables the given Input Port * * @param port */ void disableInputPort(Port port); /** - * Starts the given Output Port + * Disables the given Output Port * * @param port */ void disableOutputPort(Port port); - /** - * Starts the given Funnel - * - * @param funnel - */ - void disableFunnel(Funnel funnel); /** * Indicates that the Flow is being shutdown; allows cleanup of resources @@ -618,11 +606,23 @@ public interface ProcessGroup { Port getOutputPortByName(String name); /** - * Adds the given funnel to this ProcessGroup + * Adds the given funnel to this ProcessGroup and starts it. While other components + * do not automatically start, the funnel does by default because it is intended to be + * more of a notional component that users are unable to explicitly start and stop. + * However, there is an override available in {@link #addFunnel(Funnel, boolean)} because + * we may need to avoid starting the funnel on restart until the flow is completely + * initialized. * * @param funnel */ void addFunnel(Funnel funnel); + + /** + * Adds the given funnel to this ProcessGroup and optionally starts the funnel. + * @param funnel + * @param autoStart + */ + void addFunnel(Funnel funnel, boolean autoStart); /** * Returns a Set of all Funnels that belong to this ProcessGroup diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java index 54f0807848..06ef203644 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java @@ -576,40 +576,57 @@ public class FlowController implements EventAccess, ControllerServiceProvider, H * flag of true to now start *

    */ - public void startDelayed() { + public void onFlowInitialized(final boolean startDelayedComponents) { writeLock.lock(); try { - LOG.info("Starting {} processors/ports/funnels", (startConnectablesAfterInitialization.size() + startRemoteGroupPortsAfterInitialization.size())); - for (final Connectable connectable : startConnectablesAfterInitialization) { - if (connectable.getScheduledState() == ScheduledState.DISABLED) { - continue; - } - - try { - if (connectable instanceof ProcessorNode) { - connectable.getProcessGroup().startProcessor((ProcessorNode) connectable); - } else { - startConnectable(connectable); + if ( startDelayedComponents ) { + LOG.info("Starting {} processors/ports/funnels", (startConnectablesAfterInitialization.size() + startRemoteGroupPortsAfterInitialization.size())); + for (final Connectable connectable : startConnectablesAfterInitialization) { + if (connectable.getScheduledState() == ScheduledState.DISABLED) { + continue; + } + + try { + if (connectable instanceof ProcessorNode) { + connectable.getProcessGroup().startProcessor((ProcessorNode) connectable); + } else { + startConnectable(connectable); + } + } catch (final Throwable t) { + LOG.error("Unable to start {} due to {}", new Object[]{connectable, t}); } - } catch (final Throwable t) { - LOG.error("Unable to start {} due to {}", new Object[]{connectable, t}); } - } - - startConnectablesAfterInitialization.clear(); - - int startedTransmitting = 0; - for (final RemoteGroupPort remoteGroupPort : startRemoteGroupPortsAfterInitialization) { - try { - remoteGroupPort.getRemoteProcessGroup().startTransmitting(remoteGroupPort); - startedTransmitting++; - } catch (final Throwable t) { - LOG.error("Unable to start transmitting with {} due to {}", new Object[]{remoteGroupPort, t}); + + startConnectablesAfterInitialization.clear(); + + int startedTransmitting = 0; + for (final RemoteGroupPort remoteGroupPort : startRemoteGroupPortsAfterInitialization) { + try { + remoteGroupPort.getRemoteProcessGroup().startTransmitting(remoteGroupPort); + startedTransmitting++; + } catch (final Throwable t) { + LOG.error("Unable to start transmitting with {} due to {}", new Object[]{remoteGroupPort, t}); + } } + + LOG.info("Started {} Remote Group Ports transmitting", startedTransmitting); + startRemoteGroupPortsAfterInitialization.clear(); + } else { + // We don't want to start all of the delayed components. However, funnels need to be started anyway + // because we don't provide users the ability to start or stop them - they are just notional. + for (final Connectable connectable : startConnectablesAfterInitialization) { + try { + if (connectable instanceof Funnel) { + startConnectable(connectable); + } + } catch (final Throwable t) { + LOG.error("Unable to start {} due to {}", new Object[]{connectable, t}); + } + } + + startConnectablesAfterInitialization.clear(); + startRemoteGroupPortsAfterInitialization.clear(); } - - LOG.info("Started {} Remote Group Ports transmitting", startedTransmitting); - startRemoteGroupPortsAfterInitialization.clear(); } finally { writeLock.unlock(); } diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardFlowService.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardFlowService.java index d459b0047c..64ce5c4142 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardFlowService.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardFlowService.java @@ -423,18 +423,15 @@ public class StandardFlowService implements FlowService, ProtocolHandler { */ controller.startHeartbeating(); - // if configured, start all components - if (autoResumeState) { - try { - controller.startDelayed(); - } catch (final Exception ex) { - logger.warn("Unable to start all processors due to invalid flow configuration."); - if (logger.isDebugEnabled()) { - logger.warn(StringUtils.EMPTY, ex); - } + // notify controller that flow is initialized + try { + controller.onFlowInitialized(autoResumeState); + } catch (final Exception ex) { + logger.warn("Unable to start all processors due to invalid flow configuration."); + if (logger.isDebugEnabled()) { + logger.warn(StringUtils.EMPTY, ex); } } - } else { try { loadFromConnectionResponse(response); @@ -732,9 +729,7 @@ public class StandardFlowService implements FlowService, ProtocolHandler { controller.setPrimary(response.isPrimary()); // start the processors as indicated by the dataflow - if (dataFlow.isAutoStartProcessors()) { - controller.startDelayed(); - } + controller.onFlowInitialized(dataFlow.isAutoStartProcessors()); loadTemplates(dataFlow.getTemplates()); loadSnippets(dataFlow.getSnippets()); diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardFlowSynchronizer.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardFlowSynchronizer.java index b60d18711f..05a8f019d8 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardFlowSynchronizer.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardFlowSynchronizer.java @@ -692,7 +692,11 @@ public class StandardFlowSynchronizer implements FlowSynchronizer { final FunnelDTO funnelDTO = FlowFromDOMFactory.getFunnel(funnelElement); final Funnel funnel = controller.createFunnel(funnelDTO.getId()); funnel.setPosition(toPosition(funnelDTO.getPosition())); - processGroup.addFunnel(funnel); + + // Since this is called during startup, we want to add the funnel without enabling it + // and then tell the controller to enable it. This way, if the controller is not fully + // initialized, the starting of the funnel is delayed until the controller is ready. + processGroup.addFunnel(funnel, false); controller.startConnectable(funnel); } diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/groups/StandardProcessGroup.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/groups/StandardProcessGroup.java index 856ccc1273..0025caa550 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/groups/StandardProcessGroup.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/groups/StandardProcessGroup.java @@ -1036,9 +1036,7 @@ public final class StandardProcessGroup implements ProcessGroup { } final ScheduledState state = funnel.getScheduledState(); - if (state == ScheduledState.DISABLED) { - throw new IllegalStateException("Funnel is disabled"); - } else if (state == ScheduledState.RUNNING) { + if (state == ScheduledState.RUNNING) { return; } scheduler.startFunnel(funnel); @@ -1131,27 +1129,6 @@ public final class StandardProcessGroup implements ProcessGroup { } } - @Override - public void enableFunnel(final Funnel funnel) { - readLock.lock(); - try { - if (!funnels.containsKey(funnel.getIdentifier())) { - throw new IllegalStateException("No Funnel with ID " + funnel.getIdentifier() + " belongs to this Process Group"); - } - - final ScheduledState state = funnel.getScheduledState(); - if (state == ScheduledState.STOPPED) { - return; - } else if (state == ScheduledState.RUNNING) { - throw new IllegalStateException("Funnel is currently running"); - } - - scheduler.enableFunnel(funnel); - } finally { - readLock.unlock(); - } - } - @Override public void enableInputPort(final Port port) { readLock.lock(); @@ -1215,26 +1192,6 @@ public final class StandardProcessGroup implements ProcessGroup { } } - @Override - public void disableFunnel(final Funnel funnel) { - readLock.lock(); - try { - if (!funnels.containsKey(funnel.getIdentifier())) { - throw new IllegalStateException("No Funnel with ID " + funnel.getIdentifier() + " belongs to this Process Group"); - } - - final ScheduledState state = funnel.getScheduledState(); - if (state == ScheduledState.DISABLED) { - return; - } else if (state == ScheduledState.RUNNING) { - throw new IllegalStateException("Funnel is currently running"); - } - - scheduler.disableFunnel(funnel); - } finally { - readLock.unlock(); - } - } @Override public void disableInputPort(final Port port) { @@ -1546,8 +1503,14 @@ public final class StandardProcessGroup implements ProcessGroup { return null; } + @Override public void addFunnel(final Funnel funnel) { + addFunnel(funnel, true); + } + + @Override + public void addFunnel(final Funnel funnel, final boolean autoStart) { writeLock.lock(); try { final Funnel existing = funnels.get(requireNonNull(funnel).getIdentifier()); @@ -1557,6 +1520,10 @@ public final class StandardProcessGroup implements ProcessGroup { funnel.setProcessGroup(this); funnels.put(funnel.getIdentifier(), funnel); + + if ( autoStart ) { + startFunnel(funnel); + } } finally { writeLock.unlock(); } diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/contextlistener/ApplicationStartupContextListener.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/contextlistener/ApplicationStartupContextListener.java index a6316701e7..8b48abf323 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/contextlistener/ApplicationStartupContextListener.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/contextlistener/ApplicationStartupContextListener.java @@ -91,10 +91,8 @@ public class ApplicationStartupContextListener implements ServletContextListener * reloading actions, the node will start the necessary * processors. */ - if (properties.getAutoResumeState()) { - final FlowController flowController = flowService.getController(); - flowController.startDelayed(); - } + final FlowController flowController = flowService.getController(); + flowController.onFlowInitialized(properties.getAutoResumeState()); logger.info("Flow Controller started successfully."); } From c9eb237895a94a27fc6f760b82b16ac3e2cdab95 Mon Sep 17 00:00:00 2001 From: Mark Payne Date: Thu, 19 Mar 2015 11:40:01 -0400 Subject: [PATCH 037/116] NIFI-443: Removed stopFunnel from ProcessGroup because we don't want to allow it --- .../src/main/java/org/apache/nifi/groups/ProcessGroup.java | 6 ------ .../java/org/apache/nifi/groups/StandardProcessGroup.java | 3 +-- 2 files changed, 1 insertion(+), 8 deletions(-) diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/groups/ProcessGroup.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/groups/ProcessGroup.java index 53b26e1787..b898638bf6 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/groups/ProcessGroup.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/groups/ProcessGroup.java @@ -192,12 +192,6 @@ public interface ProcessGroup { */ void stopOutputPort(Port port); - /** - * Stops the given Funnel - * - * @param processor - */ - void stopFunnel(Funnel funnel); /** * Disables the given Processor diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/groups/StandardProcessGroup.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/groups/StandardProcessGroup.java index 0025caa550..3cd5853265 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/groups/StandardProcessGroup.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/groups/StandardProcessGroup.java @@ -1108,8 +1108,7 @@ public final class StandardProcessGroup implements ProcessGroup { } } - @Override - public void stopFunnel(final Funnel funnel) { + private void stopFunnel(final Funnel funnel) { readLock.lock(); try { if (!funnels.containsKey(funnel.getIdentifier())) { From 3ac9cbaa21c5c53a575f4086b2d7456ca22bfa17 Mon Sep 17 00:00:00 2001 From: danbress Date: Thu, 5 Mar 2015 11:04:16 -0500 Subject: [PATCH 038/116] NIFI-309 - Adding support for Reads/Writes Attributes --- .../documentation/ReadsAttribute.java | 41 +++++++ .../documentation/ReadsAttributes.java | 39 ++++++ .../annotation/documentation/SeeAlso.java | 43 +++++++ .../documentation/WritesAttribute.java | 41 +++++++ .../documentation/WritesAttributes.java | 40 ++++++ .../HtmlProcessorDocumentationWriter.java | 115 +++++++++++++++++- .../example/FullyDocumentedProcessor.java | 9 +- .../ProcessorDocumentationWriterTest.java | 1 + 8 files changed, 325 insertions(+), 4 deletions(-) create mode 100644 nifi/nifi-api/src/main/java/org/apache/nifi/annotation/documentation/ReadsAttribute.java create mode 100644 nifi/nifi-api/src/main/java/org/apache/nifi/annotation/documentation/ReadsAttributes.java create mode 100644 nifi/nifi-api/src/main/java/org/apache/nifi/annotation/documentation/SeeAlso.java create mode 100644 nifi/nifi-api/src/main/java/org/apache/nifi/annotation/documentation/WritesAttribute.java create mode 100644 nifi/nifi-api/src/main/java/org/apache/nifi/annotation/documentation/WritesAttributes.java diff --git a/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/documentation/ReadsAttribute.java b/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/documentation/ReadsAttribute.java new file mode 100644 index 0000000000..d644d94df4 --- /dev/null +++ b/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/documentation/ReadsAttribute.java @@ -0,0 +1,41 @@ +/* + * 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.annotation.documentation; + +import java.lang.annotation.Documented; +import java.lang.annotation.ElementType; +import java.lang.annotation.Inherited; +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; +import java.lang.annotation.Target; + +/** + * Annotation that may be placed on a {@link org.apache.nifi.processor.Processor Processor} + * indicating that this processor reads a specific FlowFile attribute. + * + * @author + * + */ + +@Documented +@Target({ElementType.TYPE}) +@Retention(RetentionPolicy.RUNTIME) +@Inherited +public @interface ReadsAttribute { + public String attribute(); + public String description() default ""; +} diff --git a/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/documentation/ReadsAttributes.java b/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/documentation/ReadsAttributes.java new file mode 100644 index 0000000000..9edd06fc8f --- /dev/null +++ b/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/documentation/ReadsAttributes.java @@ -0,0 +1,39 @@ +/* + * 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.annotation.documentation; + +import java.lang.annotation.Documented; +import java.lang.annotation.ElementType; +import java.lang.annotation.Inherited; +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; +import java.lang.annotation.Target; + +/** + * Annotation that may be placed on a {@link org.apache.nifi.processor.Processor Processor} + * indicating that this processor reads specific FlowFile attributes. + * + * @author + * + */ +@Documented +@Target({ElementType.TYPE}) +@Retention(RetentionPolicy.RUNTIME) +@Inherited +public @interface ReadsAttributes { + public ReadsAttribute[] value(); +} diff --git a/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/documentation/SeeAlso.java b/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/documentation/SeeAlso.java new file mode 100644 index 0000000000..78daa58f82 --- /dev/null +++ b/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/documentation/SeeAlso.java @@ -0,0 +1,43 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.annotation.documentation; + +import java.lang.annotation.Documented; +import java.lang.annotation.ElementType; +import java.lang.annotation.Inherited; +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; +import java.lang.annotation.Target; + +import org.apache.nifi.components.ConfigurableComponent; + +/** + * Annotation that may be placed on a {@link org.apache.nifi.processor.Processor Processor}, + * {@link org.apache.nifi.controller.ControllerService ControllerService}, or + * {@link org.apache.nifi.reporting.ReportingTask ReportingTask} that indicates this component is related + * to the components listed. + * @author + * + */ + +@Documented +@Target({ElementType.TYPE}) +@Retention(RetentionPolicy.RUNTIME) +@Inherited +public @interface SeeAlso { + public Class[] value(); +} diff --git a/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/documentation/WritesAttribute.java b/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/documentation/WritesAttribute.java new file mode 100644 index 0000000000..16ef56b8db --- /dev/null +++ b/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/documentation/WritesAttribute.java @@ -0,0 +1,41 @@ +/* + * 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.annotation.documentation; + +import java.lang.annotation.Documented; +import java.lang.annotation.ElementType; +import java.lang.annotation.Inherited; +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; +import java.lang.annotation.Target; + +/** + * Annotation that may be placed on a {@link org.apache.nifi.processor.Processor Processor} + * indicating that this processor writes/updates a specific FlowFile attribute. + * + * @author + * + */ + +@Documented +@Target({ElementType.TYPE}) +@Retention(RetentionPolicy.RUNTIME) +@Inherited +public @interface WritesAttribute { + public String attribute(); + public String description() default ""; +} diff --git a/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/documentation/WritesAttributes.java b/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/documentation/WritesAttributes.java new file mode 100644 index 0000000000..d9d238028a --- /dev/null +++ b/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/documentation/WritesAttributes.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.annotation.documentation; + +import java.lang.annotation.Documented; +import java.lang.annotation.ElementType; +import java.lang.annotation.Inherited; +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; +import java.lang.annotation.Target; + +/** + * Annotation that may be placed on a {@link org.apache.nifi.processor.Processor Processor} + * indicating that this processor writes/updates specific FlowFile attributes. + * + * @author + * + */ + +@Documented +@Target({ElementType.TYPE}) +@Retention(RetentionPolicy.RUNTIME) +@Inherited +public @interface WritesAttributes { + public WritesAttribute[] value(); +} 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 412ebf6d43..74615f571c 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 @@ -16,25 +16,136 @@ */ package org.apache.nifi.documentation.html; +import java.util.ArrayList; +import java.util.List; + import javax.xml.stream.XMLStreamException; import javax.xml.stream.XMLStreamWriter; +import org.apache.commons.lang3.StringUtils; +import org.apache.nifi.annotation.documentation.ReadsAttribute; +import org.apache.nifi.annotation.documentation.ReadsAttributes; +import org.apache.nifi.annotation.documentation.WritesAttribute; +import org.apache.nifi.annotation.documentation.WritesAttributes; 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. - * + * 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; + writeRelationships(processor, xmlStreamWriter); + writeAttributeInfo(processor, xmlStreamWriter); + } + + private void writeAttributeInfo(ConfigurableComponent configurableComponent, + XMLStreamWriter xmlStreamWriter) throws XMLStreamException { final Processor processor = (Processor) configurableComponent; + + handleReadsAttributes(xmlStreamWriter, processor); + handleWritesAttributes(xmlStreamWriter, processor); + } + + private void handleReadsAttributes(XMLStreamWriter xmlStreamWriter, final Processor processor) + throws XMLStreamException { + List attributesRead = getReadsAttributes(processor); + + writeSimpleElement(xmlStreamWriter, "h3", "Reads Attributes: "); + if (attributesRead.size() > 0) { + xmlStreamWriter.writeStartElement("table"); + xmlStreamWriter.writeStartElement("tr"); + writeSimpleElement(xmlStreamWriter, "th", "Name"); + writeSimpleElement(xmlStreamWriter, "th", "Description"); + xmlStreamWriter.writeEndElement(); + for (ReadsAttribute attribute : attributesRead) { + xmlStreamWriter.writeStartElement("tr"); + writeSimpleElement(xmlStreamWriter, "td", + StringUtils.defaultIfBlank(attribute.attribute(), "Not Specified")); + writeSimpleElement(xmlStreamWriter, "td", + StringUtils.defaultIfBlank(attribute.description(), "Not Specified")); + xmlStreamWriter.writeEndElement(); + } + xmlStreamWriter.writeEndElement(); + + } else { + xmlStreamWriter.writeCharacters("None specified."); + } + } + + private void handleWritesAttributes(XMLStreamWriter xmlStreamWriter, final Processor processor) + throws XMLStreamException { + List attributesRead = getWritesAttributes(processor); + + writeSimpleElement(xmlStreamWriter, "h3", "Writes Attributes: "); + if (attributesRead.size() > 0) { + xmlStreamWriter.writeStartElement("table"); + xmlStreamWriter.writeStartElement("tr"); + writeSimpleElement(xmlStreamWriter, "th", "Name"); + writeSimpleElement(xmlStreamWriter, "th", "Description"); + xmlStreamWriter.writeEndElement(); + for (WritesAttribute attribute : attributesRead) { + xmlStreamWriter.writeStartElement("tr"); + writeSimpleElement(xmlStreamWriter, "td", + StringUtils.defaultIfBlank(attribute.attribute(), "Not Specified")); + writeSimpleElement(xmlStreamWriter, "td", + StringUtils.defaultIfBlank(attribute.description(), "Not Specified")); + xmlStreamWriter.writeEndElement(); + } + xmlStreamWriter.writeEndElement(); + + } else { + xmlStreamWriter.writeCharacters("None specified."); + } + } + + private List getReadsAttributes(Processor processor) { + List attributes = new ArrayList<>(); + + ReadsAttributes readsAttributes = processor.getClass().getAnnotation(ReadsAttributes.class); + if (readsAttributes != null) { + for (ReadsAttribute readAttribute : readsAttributes.value()) { + attributes.add(readAttribute); + } + } + + ReadsAttribute readsAttribute = processor.getClass().getAnnotation(ReadsAttribute.class); + if (readsAttribute != null) { + attributes.add(readsAttribute); + } + + return attributes; + } + + private List getWritesAttributes(Processor processor) { + List attributes = new ArrayList<>(); + + WritesAttributes writesAttributes = processor.getClass().getAnnotation(WritesAttributes.class); + if (writesAttributes != null) { + for (WritesAttribute writeAttribute : writesAttributes.value()) { + attributes.add(writeAttribute); + } + } + + WritesAttribute writeAttribute = processor.getClass().getAnnotation(WritesAttribute.class); + if (writeAttribute != null) { + attributes.add(writeAttribute); + } + + return attributes; + } + + private void writeRelationships(final Processor processor, final XMLStreamWriter xmlStreamWriter) + throws XMLStreamException { + writeSimpleElement(xmlStreamWriter, "h3", "Relationships: "); if (processor.getRelationships().size() > 0) { 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 c2c765784f..20307e806a 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,11 @@ import java.util.HashSet; import java.util.List; import java.util.Set; +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.documentation.ReadsAttribute; +import org.apache.nifi.annotation.documentation.Tags; +import org.apache.nifi.annotation.documentation.WritesAttribute; +import org.apache.nifi.annotation.documentation.WritesAttributes; import org.apache.nifi.components.AllowableValue; import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.processor.AbstractProcessor; @@ -29,13 +34,13 @@ 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.") +@WritesAttributes({@WritesAttribute(attribute="first", description="this is the first attribute i write"), @WritesAttribute(attribute="second")}) +@ReadsAttribute(attribute = "incoming", description="this specifies the format of the thing") public class FullyDocumentedProcessor extends AbstractProcessor { public static final PropertyDescriptor DIRECTORY = new PropertyDescriptor.Builder().name("Input Directory") 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 5306ddf2ee..ffd31d8b9f 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 @@ -69,6 +69,7 @@ public class ProcessorDocumentationWriterTest { assertNotContains(results, "No description provided."); assertNotContains(results, "No Tags provided."); assertNotContains(results, "Additional Details..."); + System.out.println(results); } @Test From faf976ca91d4e083d28a520974c98dff1acc1905 Mon Sep 17 00:00:00 2001 From: danbress Date: Thu, 5 Mar 2015 11:20:51 -0500 Subject: [PATCH 039/116] NIFI-309 - Adding support or SeeAlso --- .../html/HtmlDocumentationWriter.java | 213 +++++++++++------- .../example/FullyDocumentedProcessor.java | 2 + 2 files changed, 139 insertions(+), 76 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 b547934d0d..1beeb9fea9 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 @@ -27,6 +27,7 @@ import javax.xml.stream.XMLStreamWriter; import org.apache.commons.lang3.StringUtils; import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.documentation.SeeAlso; import org.apache.nifi.annotation.documentation.Tags; import org.apache.nifi.components.AllowableValue; import org.apache.nifi.components.ConfigurableComponent; @@ -68,12 +69,14 @@ public class HtmlDocumentationWriter implements DocumentationWriter { } /** - * 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 + * 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 { @@ -93,9 +96,10 @@ public class HtmlDocumentationWriter implements DocumentationWriter { } /** - * Gets the class name of the component. - * - * @param configurableComponent the component to describe + * 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) { @@ -104,42 +108,80 @@ public class HtmlDocumentationWriter implements DocumentationWriter { /** * 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 + * 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, + 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); + writeAdditionalBodyInfo(configurableComponent, xmlStreamWriter); + writeSeeAlso(configurableComponent, xmlStreamWriter); xmlStreamWriter.writeEndElement(); } + /** + * Writes the list of components that may be linked from this component. + * + * @param configurableComponent + * the component to describe + * @param xmlStreamWriter + * the stream writer to use + * @throws XMLStreamException + * thrown if there was a problem writing the XML + */ + private void writeSeeAlso(ConfigurableComponent configurableComponent, XMLStreamWriter xmlStreamWriter) + throws XMLStreamException { + final SeeAlso seeAlso = configurableComponent.getClass().getAnnotation(SeeAlso.class); + if (seeAlso != null && seeAlso.value().length > 0) { + writeSimpleElement(xmlStreamWriter, "h3", "See Also:"); + xmlStreamWriter.writeStartElement("p"); + int index = 0; + for (final Class linkedComponent : seeAlso.value()) { + if (index != 0) { + xmlStreamWriter.writeCharacters(", "); + } + + final String link = "../" + linkedComponent.getCanonicalName() + "/index.html"; + + writeLink(xmlStreamWriter, linkedComponent.getSimpleName(), link); + + ++index; + } + 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 + * + * @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. - * + * Writes the tags attached to a ConfigurableComponent. + * * @param configurableComponent * @param xmlStreamWriter * @throws XMLStreamException @@ -162,14 +204,17 @@ public class HtmlDocumentationWriter implements DocumentationWriter { } /** - * 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 + * 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) @@ -187,9 +232,10 @@ public class HtmlDocumentationWriter implements DocumentationWriter { /** * Gets a description of the ConfigurableComponent using the - * CapabilityDescription annotation. - * - * @param configurableComponent the component to describe + * CapabilityDescription annotation. + * + * @param configurableComponent + * the component to describe * @return a description of the configurableComponent */ protected String getDescription(final ConfigurableComponent configurableComponent) { @@ -207,12 +253,14 @@ public class HtmlDocumentationWriter implements DocumentationWriter { } /** - * 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 + * 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 { @@ -222,9 +270,8 @@ public class HtmlDocumentationWriter implements DocumentationWriter { 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"); + + "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 "); @@ -303,12 +350,14 @@ public class HtmlDocumentationWriter implements DocumentationWriter { /** * 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 + * 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 { @@ -325,8 +374,10 @@ public class HtmlDocumentationWriter implements DocumentationWriter { } xmlStreamWriter.writeEndElement(); - } else if (property.getControllerServiceDefinition() != null) { - Class controllerServiceClass = property.getControllerServiceDefinition(); + } else if (property.getControllerServiceDefinition() != null) { + Class controllerServiceClass = property + .getControllerServiceDefinition(); + writeSimpleElement(xmlStreamWriter, "strong", "Controller Service: "); xmlStreamWriter.writeEmptyElement("br"); xmlStreamWriter.writeCharacters(controllerServiceClass.getSimpleName()); @@ -335,14 +386,18 @@ public class HtmlDocumentationWriter implements DocumentationWriter { /** * 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. + * 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 { @@ -359,13 +414,16 @@ public class HtmlDocumentationWriter implements DocumentationWriter { /** * 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 + * 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 { @@ -373,13 +431,16 @@ public class HtmlDocumentationWriter implements DocumentationWriter { } /** - * 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 + * 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 { 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 20307e806a..b9bce9123a 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 @@ -24,6 +24,7 @@ import java.util.Set; import org.apache.nifi.annotation.documentation.CapabilityDescription; import org.apache.nifi.annotation.documentation.ReadsAttribute; +import org.apache.nifi.annotation.documentation.SeeAlso; import org.apache.nifi.annotation.documentation.Tags; import org.apache.nifi.annotation.documentation.WritesAttribute; import org.apache.nifi.annotation.documentation.WritesAttributes; @@ -41,6 +42,7 @@ import org.apache.nifi.processor.util.StandardValidators; @CapabilityDescription("This is a processor that is used to test documentation.") @WritesAttributes({@WritesAttribute(attribute="first", description="this is the first attribute i write"), @WritesAttribute(attribute="second")}) @ReadsAttribute(attribute = "incoming", description="this specifies the format of the thing") +@SeeAlso({FullyDocumentedControllerService.class, FullyDocumentedReportingTask.class}) public class FullyDocumentedProcessor extends AbstractProcessor { public static final PropertyDescriptor DIRECTORY = new PropertyDescriptor.Builder().name("Input Directory") From 9288f237e0e15bc7cf9af67ec7c1dfd96e5dd3dd Mon Sep 17 00:00:00 2001 From: danbress Date: Thu, 5 Mar 2015 11:29:34 -0500 Subject: [PATCH 040/116] NIFI-309 - writing javadoc --- .../HtmlProcessorDocumentationWriter.java | 61 +++++++++++++++++-- 1 file changed, 56 insertions(+), 5 deletions(-) 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 74615f571c..f83b2f69d9 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 @@ -47,14 +47,32 @@ public class HtmlProcessorDocumentationWriter extends HtmlDocumentationWriter { writeAttributeInfo(processor, xmlStreamWriter); } - private void writeAttributeInfo(ConfigurableComponent configurableComponent, - XMLStreamWriter xmlStreamWriter) throws XMLStreamException { - final Processor processor = (Processor) configurableComponent; + /** + * Writes all the attributes that a processor says it reads and writes + * + * @param processor + * the processor to describe + * @param xmlStreamWriter + * the xml stream writer to use + * @throws XMLStreamException + * thrown if there was a problem writing the XML + */ + private void writeAttributeInfo(Processor processor, XMLStreamWriter xmlStreamWriter) + throws XMLStreamException { handleReadsAttributes(xmlStreamWriter, processor); handleWritesAttributes(xmlStreamWriter, processor); } + /** + * Writes out just the attributes that are being read in a table form. + * + * @param xmlStreamWriter + * the xml stream writer to use + * @param processor + * the processor to describe + * @throws XMLStreamException + */ private void handleReadsAttributes(XMLStreamWriter xmlStreamWriter, final Processor processor) throws XMLStreamException { List attributesRead = getReadsAttributes(processor); @@ -81,6 +99,15 @@ public class HtmlProcessorDocumentationWriter extends HtmlDocumentationWriter { } } + /** + * Writes out just the attributes that are being written to in a table form. + * + * @param xmlStreamWriter + * the xml stream writer to use + * @param processor + * the processor to describe + * @throws XMLStreamException + */ private void handleWritesAttributes(XMLStreamWriter xmlStreamWriter, final Processor processor) throws XMLStreamException { List attributesRead = getWritesAttributes(processor); @@ -107,6 +134,13 @@ public class HtmlProcessorDocumentationWriter extends HtmlDocumentationWriter { } } + /** + * Collects the attributes that a processor is reading from. + * + * @param processor + * the processor to describe + * @return the list of attributes that processor is reading + */ private List getReadsAttributes(Processor processor) { List attributes = new ArrayList<>(); @@ -116,7 +150,7 @@ public class HtmlProcessorDocumentationWriter extends HtmlDocumentationWriter { attributes.add(readAttribute); } } - + ReadsAttribute readsAttribute = processor.getClass().getAnnotation(ReadsAttribute.class); if (readsAttribute != null) { attributes.add(readsAttribute); @@ -125,6 +159,13 @@ public class HtmlProcessorDocumentationWriter extends HtmlDocumentationWriter { return attributes; } + /** + * Collects the attributes that a processor is writing to. + * + * @param processor + * the processor to describe + * @return the list of attributes the processor is writing + */ private List getWritesAttributes(Processor processor) { List attributes = new ArrayList<>(); @@ -134,7 +175,7 @@ public class HtmlProcessorDocumentationWriter extends HtmlDocumentationWriter { attributes.add(writeAttribute); } } - + WritesAttribute writeAttribute = processor.getClass().getAnnotation(WritesAttribute.class); if (writeAttribute != null) { attributes.add(writeAttribute); @@ -143,6 +184,16 @@ public class HtmlProcessorDocumentationWriter extends HtmlDocumentationWriter { return attributes; } + /** + * Writes a table describing the relations a processor has. + * + * @param processor + * the processor to describe + * @param xmlStreamWriter + * the stream writer to use + * @throws XMLStreamException + * thrown if there was a problem writing the xml + */ private void writeRelationships(final Processor processor, final XMLStreamWriter xmlStreamWriter) throws XMLStreamException { From 376a9c6acfd6fd6dac7c35124b0763be38e619a1 Mon Sep 17 00:00:00 2001 From: danbress Date: Thu, 5 Mar 2015 12:23:25 -0500 Subject: [PATCH 041/116] NIFI-309 - Adding doc annotations to std processors --- .../annotation/documentation/SeeAlso.java | 28 +++++++++++--- .../html/HtmlDocumentationWriter.java | 14 +++++++ .../processors/standard/CompressContent.java | 4 ++ .../processors/standard/DetectDuplicate.java | 15 +++++--- .../processors/standard/EvaluateXPath.java | 2 + .../processors/standard/EvaluateXQuery.java | 19 +++++----- .../standard/ExecuteStreamCommand.java | 25 ++++++++----- .../nifi/processors/standard/GetFTP.java | 12 ++++++ .../nifi/processors/standard/GetFile.java | 21 +++++++++-- .../nifi/processors/standard/GetHTTP.java | 8 ++-- .../nifi/processors/standard/GetJMSQueue.java | 2 + .../nifi/processors/standard/GetJMSTopic.java | 2 + .../nifi/processors/standard/GetSFTP.java | 17 +++++++-- .../processors/standard/HashAttribute.java | 17 +++++---- .../nifi/processors/standard/HashContent.java | 2 + .../processors/standard/IdentifyMimeType.java | 13 ++++--- .../nifi/processors/standard/InvokeHTTP.java | 18 ++++++--- .../processors/standard/MergeContent.java | 17 +++++++++ .../processors/standard/MonitorActivity.java | 15 +++++--- .../nifi/processors/standard/PostHTTP.java | 12 +++--- .../nifi/processors/standard/PutFTP.java | 10 +++-- .../nifi/processors/standard/PutFile.java | 8 ++-- .../nifi/processors/standard/PutJMS.java | 6 ++- .../nifi/processors/standard/PutSFTP.java | 8 ++-- .../nifi/processors/standard/ScanContent.java | 18 +++++---- .../processors/standard/SegmentContent.java | 12 ++++++ .../processors/standard/SplitContent.java | 23 ++++++++---- .../nifi/processors/standard/SplitText.java | 10 +++++ .../processors/standard/UnpackContent.java | 37 ++++++++++++------- 29 files changed, 285 insertions(+), 110 deletions(-) diff --git a/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/documentation/SeeAlso.java b/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/documentation/SeeAlso.java index 78daa58f82..3ad8081984 100644 --- a/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/documentation/SeeAlso.java +++ b/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/documentation/SeeAlso.java @@ -26,18 +26,34 @@ import java.lang.annotation.Target; import org.apache.nifi.components.ConfigurableComponent; /** - * Annotation that may be placed on a {@link org.apache.nifi.processor.Processor Processor}, + * Annotation that may be placed on a + * {@link org.apache.nifi.processor.Processor Processor}, * {@link org.apache.nifi.controller.ControllerService ControllerService}, or - * {@link org.apache.nifi.reporting.ReportingTask ReportingTask} that indicates this component is related - * to the components listed. - * @author + * {@link org.apache.nifi.reporting.ReportingTask ReportingTask} that indicates + * this component is related to the components listed. + * + * @author * */ @Documented -@Target({ElementType.TYPE}) +@Target({ ElementType.TYPE }) @Retention(RetentionPolicy.RUNTIME) @Inherited public @interface SeeAlso { - public Class[] value(); + /** + * Classes you want to link to. + * + * @return + */ + public Class[] value(); + + /** + * Fully qualified class names you want to link to. Use this when the class + * you want to link to is not in the class path of the component you are + * linking from. + * + * @return + */ + public String[] classNames() default ""; } 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 1beeb9fea9..1bfdd2709f 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 @@ -159,6 +159,20 @@ public class HtmlDocumentationWriter implements DocumentationWriter { ++index; } + + for (final String linkedComponent : seeAlso.classNames()) { + if (index != 0) { + xmlStreamWriter.writeCharacters(", "); + } + + final String link = "../" + linkedComponent + "/index.html"; + + final int indexOfLastPeriod = Math.min(0, linkedComponent.lastIndexOf(".")); + + writeLink(xmlStreamWriter, linkedComponent.substring(indexOfLastPeriod), link); + + ++index; + } xmlStreamWriter.writeEndElement(); } } diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/CompressContent.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/CompressContent.java index e631cd0e62..30cb091f76 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/CompressContent.java +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/CompressContent.java @@ -40,7 +40,9 @@ import org.apache.nifi.annotation.behavior.EventDriven; import org.apache.nifi.annotation.behavior.SideEffectFree; import org.apache.nifi.annotation.behavior.SupportsBatching; import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.documentation.ReadsAttribute; import org.apache.nifi.annotation.documentation.Tags; +import org.apache.nifi.annotation.documentation.WritesAttribute; import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.flowfile.FlowFile; import org.apache.nifi.flowfile.attributes.CoreAttributes; @@ -66,6 +68,8 @@ import org.tukaani.xz.XZOutputStream; @SupportsBatching @Tags({"content", "compress", "decompress", "gzip", "bzip2", "lzma", "xz-lzma2"}) @CapabilityDescription("Compresses or decompresses the contents of FlowFiles using a user-specified compression algorithm and updates the mime.type attribute as appropriate") +@ReadsAttribute(attribute="mime.type", description="If the Compression Format is set to use mime.type attribute, this attribute is used to determine the compression type. Otherwise, this attribute is ignored.") +@WritesAttribute(attribute="mime.type", description="If 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.") public class CompressContent extends AbstractProcessor { public static final String COMPRESSION_FORMAT_ATTRIBUTE = "use mime.type attribute"; diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/DetectDuplicate.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/DetectDuplicate.java index 8910fdcd31..f2a9e3ed56 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/DetectDuplicate.java +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/DetectDuplicate.java @@ -26,6 +26,13 @@ import java.util.List; import java.util.Set; import java.util.concurrent.TimeUnit; +import org.apache.commons.lang3.StringUtils; +import org.apache.nifi.annotation.behavior.EventDriven; +import org.apache.nifi.annotation.behavior.SupportsBatching; +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.documentation.SeeAlso; +import org.apache.nifi.annotation.documentation.Tags; +import org.apache.nifi.annotation.documentation.WritesAttribute; import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.distributed.cache.client.Deserializer; import org.apache.nifi.distributed.cache.client.DistributedMapCacheClient; @@ -39,15 +46,9 @@ import org.apache.nifi.processor.AbstractProcessor; import org.apache.nifi.processor.ProcessContext; import org.apache.nifi.processor.ProcessSession; import org.apache.nifi.processor.Relationship; -import org.apache.nifi.annotation.documentation.CapabilityDescription; -import org.apache.nifi.annotation.behavior.EventDriven; -import org.apache.nifi.annotation.behavior.SupportsBatching; -import org.apache.nifi.annotation.documentation.Tags; import org.apache.nifi.processor.exception.ProcessException; import org.apache.nifi.processor.util.StandardValidators; -import org.apache.commons.lang3.StringUtils; - @EventDriven @SupportsBatching @Tags({"experimental", "hash", "dupe", "duplicate", "dedupe"}) @@ -55,6 +56,8 @@ import org.apache.commons.lang3.StringUtils; + "If so, routes the FlowFile to 'duplicate' with an attribute named 'original.identifier' that specifies the original FlowFile's" + "\"description\", which is specified in the property. If the FlowFile is not determined to be a duplicate, the Processor " + "routes the FlowFile to 'non-duplicate'") +@WritesAttribute(attribute="original.flowfile.description", description="All 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.") +@SeeAlso(value={DistributedMapCacheClient.class}, classNames={"org.apache.nifi.distributed.cache.server.map.DistributedMapCacheServer"}) public class DetectDuplicate extends AbstractProcessor { public static final String ORIGINAL_DESCRIPTION_ATTRIBUTE_NAME = "original.flowfile.description"; diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateXPath.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateXPath.java index f4d2d34fee..d7ca00a581 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateXPath.java +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateXPath.java @@ -66,6 +66,7 @@ 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.WritesAttribute; import org.apache.nifi.annotation.behavior.EventDriven; import org.apache.nifi.annotation.lifecycle.OnScheduled; import org.apache.nifi.annotation.behavior.SideEffectFree; @@ -91,6 +92,7 @@ import org.xml.sax.InputSource; + "evaluate to a Node, 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'") +@WritesAttribute(attribute="user-defined", description="This processor adds user-defined attributes if the property is set to flowfile-attribute.") public class EvaluateXPath extends AbstractProcessor { public static final String DESTINATION_ATTRIBUTE = "flowfile-attribute"; diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateXQuery.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateXQuery.java index 3ddee83d81..314ff9989e 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateXQuery.java +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateXQuery.java @@ -49,30 +49,31 @@ import net.sf.saxon.s9api.XQueryExecutable; import net.sf.saxon.s9api.XdmItem; import net.sf.saxon.s9api.XdmNode; import net.sf.saxon.s9api.XdmValue; + +import org.apache.nifi.annotation.behavior.EventDriven; +import org.apache.nifi.annotation.behavior.SideEffectFree; +import org.apache.nifi.annotation.behavior.SupportsBatching; +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.documentation.Tags; +import org.apache.nifi.annotation.documentation.WritesAttribute; import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.components.ValidationContext; import org.apache.nifi.components.ValidationResult; import org.apache.nifi.components.Validator; import org.apache.nifi.flowfile.FlowFile; -import org.apache.nifi.stream.io.BufferedInputStream; -import org.apache.nifi.stream.io.BufferedOutputStream; import org.apache.nifi.logging.ProcessorLog; 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.behavior.EventDriven; -import org.apache.nifi.annotation.behavior.SideEffectFree; -import org.apache.nifi.annotation.behavior.SupportsBatching; -import org.apache.nifi.annotation.documentation.Tags; import org.apache.nifi.processor.exception.ProcessException; import org.apache.nifi.processor.io.InputStreamCallback; import org.apache.nifi.processor.io.OutputStreamCallback; import org.apache.nifi.processor.util.StandardValidators; +import org.apache.nifi.stream.io.BufferedInputStream; +import org.apache.nifi.stream.io.BufferedOutputStream; import org.apache.nifi.util.ObjectHolder; - import org.w3c.dom.Document; import org.xml.sax.InputSource; @@ -92,7 +93,7 @@ import org.xml.sax.InputSource; + "'matched'. If no provided XQuery returns a result, the FlowFile will be routed to 'unmatched'. If the " + "Destination is 'flowfile-attribute' and the XQueries matche nothing, no attributes will be applied to the " + "FlowFile.") - +@WritesAttribute(attribute="user-defined", description="This processor adds user-defined attributes if the property is set to flowfile-attribute .") public class EvaluateXQuery extends AbstractProcessor { public static final String DESTINATION_ATTRIBUTE = "flowfile-attribute"; diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ExecuteStreamCommand.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ExecuteStreamCommand.java index dda3647bbb..c2f93650b1 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ExecuteStreamCommand.java +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ExecuteStreamCommand.java @@ -31,32 +31,33 @@ import java.util.List; import java.util.Map; import java.util.Set; +import org.apache.commons.io.IOUtils; +import org.apache.commons.lang3.StringUtils; +import org.apache.nifi.annotation.behavior.EventDriven; +import org.apache.nifi.annotation.behavior.SupportsBatching; +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.documentation.Tags; +import org.apache.nifi.annotation.documentation.WritesAttribute; +import org.apache.nifi.annotation.documentation.WritesAttributes; import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.components.ValidationContext; import org.apache.nifi.components.ValidationResult; import org.apache.nifi.components.Validator; import org.apache.nifi.expression.AttributeExpression.ResultType; import org.apache.nifi.flowfile.FlowFile; -import org.apache.nifi.stream.io.BufferedInputStream; -import org.apache.nifi.stream.io.BufferedOutputStream; -import org.apache.nifi.stream.io.StreamUtils; import org.apache.nifi.logging.ProcessorLog; 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.behavior.EventDriven; -import org.apache.nifi.annotation.behavior.SupportsBatching; -import org.apache.nifi.annotation.documentation.Tags; import org.apache.nifi.processor.exception.ProcessException; import org.apache.nifi.processor.io.InputStreamCallback; import org.apache.nifi.processor.io.OutputStreamCallback; import org.apache.nifi.processor.util.StandardValidators; - -import org.apache.commons.io.IOUtils; -import org.apache.commons.lang3.StringUtils; +import org.apache.nifi.stream.io.BufferedInputStream; +import org.apache.nifi.stream.io.BufferedOutputStream; +import org.apache.nifi.stream.io.StreamUtils; /** *

    @@ -119,6 +120,10 @@ import org.apache.commons.lang3.StringUtils; @SupportsBatching @Tags({"command execution", "command", "stream", "execute"}) @CapabilityDescription("Executes an external command on the contents of a flow file, and creates a new flow file with the results of the command.") +@WritesAttributes({ @WritesAttribute(attribute = "execution.command", description = "The name of the command executed to create the new FlowFile"), + @WritesAttribute(attribute = "execution.command.args", description = "The semi-colon delimited list of arguments"), + @WritesAttribute(attribute = "execution.status", description = "The exit status code returned from executing the command"), + @WritesAttribute(attribute = "execution.error", description = "Any error messages returned from executing the command") }) public class ExecuteStreamCommand extends AbstractProcessor { public static final Relationship ORIGINAL_RELATIONSHIP = new Relationship.Builder() diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/GetFTP.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/GetFTP.java index 18bdc93995..5077b7fe76 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/GetFTP.java +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/GetFTP.java @@ -24,6 +24,9 @@ import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.processor.ProcessContext; import org.apache.nifi.processor.ProcessorInitializationContext; import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.documentation.SeeAlso; +import org.apache.nifi.annotation.documentation.WritesAttribute; +import org.apache.nifi.annotation.documentation.WritesAttributes; import org.apache.nifi.annotation.behavior.SideEffectFree; import org.apache.nifi.annotation.documentation.Tags; import org.apache.nifi.processors.standard.util.FTPTransfer; @@ -32,6 +35,15 @@ import org.apache.nifi.processors.standard.util.FileTransfer; @SideEffectFree @Tags({"FTP", "get", "retrieve", "files", "fetch", "remote", "ingest", "source", "input"}) @CapabilityDescription("Fetches files from an FTP Server and creates FlowFiles from them") +@WritesAttributes({ @WritesAttribute(attribute = "filename", description = "The filename is set to the name of the file on the remote server"), + @WritesAttribute(attribute = "path", description = "The path is set to the path of the file's directory on the remote server. For example, if the property is set to /tmp, files picked up from /tmp will have the path attribute set to /tmp. If the 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"), + @WritesAttribute(attribute = "file.lastModifiedTime", description = "The date and time that the source file was last modified"), + @WritesAttribute(attribute = "file.lastAccessTime", description = "The date and time that the file was last accessed. May not work on all file systems"), + @WritesAttribute(attribute = "file.owner", description = "The numeric owner id of the source file"), + @WritesAttribute(attribute = "file.group", description = "The numeric group id of the source file"), + @WritesAttribute(attribute = "file.permissions", description = "The read/write/execute permissions of the source file"), + @WritesAttribute(attribute = "absolute.path", description = "The full/absolute path from where a file was picked up. The current 'path' attribute is still populated, but may be a relative path")}) +@SeeAlso(PutFTP.class) public class GetFTP extends GetFileTransfer { private List properties; diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/GetFile.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/GetFile.java index 2a2504c654..881e25b2d6 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/GetFile.java +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/GetFile.java @@ -49,6 +49,13 @@ import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.ReentrantLock; import java.util.regex.Pattern; +import org.apache.nifi.annotation.behavior.TriggerWhenEmpty; +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.documentation.SeeAlso; +import org.apache.nifi.annotation.documentation.Tags; +import org.apache.nifi.annotation.documentation.WritesAttribute; +import org.apache.nifi.annotation.documentation.WritesAttributes; +import org.apache.nifi.annotation.lifecycle.OnScheduled; import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.flowfile.FlowFile; import org.apache.nifi.flowfile.attributes.CoreAttributes; @@ -59,16 +66,22 @@ 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.lifecycle.OnScheduled; -import org.apache.nifi.annotation.documentation.Tags; -import org.apache.nifi.annotation.behavior.TriggerWhenEmpty; import org.apache.nifi.processor.exception.ProcessException; import org.apache.nifi.processor.util.StandardValidators; @TriggerWhenEmpty @Tags({"local", "files", "filesystem", "ingest", "ingress", "get", "source", "input"}) @CapabilityDescription("Creates FlowFiles from files in a directory. NiFi will ignore files it doesn't have at least read permissions for.") +@WritesAttributes({ @WritesAttribute(attribute = "filename", description = "The filename is set to the name of the file on disk"), + @WritesAttribute(attribute = "path", description = "The path is set to the relative path of the file's directory on disk. For example, if the property is set to /tmp, files picked up from /tmp will have the path attribute set to ./. If the 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"), + @WritesAttribute(attribute = "file.creationTime", description = "The date and time that the file was created. May not work on all file systems"), + @WritesAttribute(attribute = "file.lastModifiedTime", description = "The date and time that the file was last modified. May not work on all file systems"), + @WritesAttribute(attribute = "file.lastAccessTime", description = "The date and time that the file was last accessed. May not work on all file systems"), + @WritesAttribute(attribute = "file.owner", description = "The owner of the file. May not work on all file systems"), + @WritesAttribute(attribute = "file.group", description = "The group owner of the file. May not work on all file systems"), + @WritesAttribute(attribute = "file.permissions", description = "The read/write/execute permissions of the file. May not work on all file systems"), + @WritesAttribute(attribute = "absolute.path", description = "The full/absolute path from where a file was picked up. The current 'path' attribute is still populated, but may be a relative path")}) +@SeeAlso(PutFile.class) public class GetFile extends AbstractProcessor { public static final PropertyDescriptor DIRECTORY = new PropertyDescriptor.Builder() diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/GetHTTP.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/GetHTTP.java index fd70024a82..1203cf9776 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/GetHTTP.java +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/GetHTTP.java @@ -67,6 +67,10 @@ import org.apache.http.conn.ssl.TrustSelfSignedStrategy; import org.apache.http.impl.client.BasicCredentialsProvider; import org.apache.http.impl.client.HttpClientBuilder; import org.apache.http.impl.conn.BasicHttpClientConnectionManager; +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.documentation.Tags; +import org.apache.nifi.annotation.documentation.WritesAttribute; +import org.apache.nifi.annotation.lifecycle.OnShutdown; import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.components.ValidationContext; import org.apache.nifi.components.ValidationResult; @@ -79,9 +83,6 @@ import org.apache.nifi.processor.ProcessSession; import org.apache.nifi.processor.ProcessSessionFactory; import org.apache.nifi.processor.ProcessorInitializationContext; import org.apache.nifi.processor.Relationship; -import org.apache.nifi.annotation.documentation.CapabilityDescription; -import org.apache.nifi.annotation.lifecycle.OnShutdown; -import org.apache.nifi.annotation.documentation.Tags; import org.apache.nifi.processor.exception.ProcessException; import org.apache.nifi.processor.util.StandardValidators; import org.apache.nifi.ssl.SSLContextService; @@ -90,6 +91,7 @@ import org.apache.nifi.util.StopWatch; @Tags({"get", "fetch", "poll", "http", "https", "ingest", "source", "input"}) @CapabilityDescription("Fetches a file via HTTP") +@WritesAttribute(attribute="filename", description="the filename is set to the name of the file on the remote server") public class GetHTTP extends AbstractSessionFactoryProcessor { static final int PERSISTENCE_INTERVAL_MSEC = 10000; diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/GetJMSQueue.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/GetJMSQueue.java index 9676d93a07..6be505a783 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/GetJMSQueue.java +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/GetJMSQueue.java @@ -23,6 +23,7 @@ import javax.jms.JMSException; import org.apache.nifi.annotation.behavior.TriggerWhenEmpty; import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.documentation.SeeAlso; import org.apache.nifi.annotation.documentation.Tags; import org.apache.nifi.annotation.lifecycle.OnStopped; import org.apache.nifi.logging.ProcessorLog; @@ -35,6 +36,7 @@ import org.apache.nifi.processors.standard.util.WrappedMessageConsumer; @TriggerWhenEmpty @Tags({"jms", "queue", "listen", "get", "pull", "source", "consume", "consumer"}) @CapabilityDescription("Pulls messages from a JMS Queue, creating a FlowFile for each JMS Message or bundle of messages, as configured") +@SeeAlso(PutJMS.class) public class GetJMSQueue extends JmsConsumer { private final Queue consumerQueue = new LinkedBlockingQueue<>(); diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/GetJMSTopic.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/GetJMSTopic.java index 8e22376268..ecec78123c 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/GetJMSTopic.java +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/GetJMSTopic.java @@ -44,6 +44,7 @@ import javax.jms.Session; import org.apache.nifi.annotation.behavior.TriggerSerially; import org.apache.nifi.annotation.behavior.TriggerWhenEmpty; import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.documentation.SeeAlso; import org.apache.nifi.annotation.documentation.Tags; import org.apache.nifi.annotation.lifecycle.OnRemoved; import org.apache.nifi.annotation.lifecycle.OnScheduled; @@ -61,6 +62,7 @@ import org.apache.nifi.processors.standard.util.WrappedMessageConsumer; @TriggerWhenEmpty @Tags({"jms", "topic", "subscription", "durable", "non-durable", "listen", "get", "pull", "source", "consume", "consumer"}) @CapabilityDescription("Pulls messages from a JMS Topic, creating a FlowFile for each JMS Message or bundle of messages, as configured") +@SeeAlso(PutJMS.class) public class GetJMSTopic extends JmsConsumer { public static final String SUBSCRIPTION_NAME_PROPERTY = "subscription.name"; diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/GetSFTP.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/GetSFTP.java index dd9f5199c8..b3c09c5982 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/GetSFTP.java +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/GetSFTP.java @@ -21,20 +21,31 @@ import java.util.Collection; import java.util.Collections; import java.util.List; +import org.apache.nifi.annotation.behavior.SideEffectFree; +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.documentation.SeeAlso; +import org.apache.nifi.annotation.documentation.Tags; +import org.apache.nifi.annotation.documentation.WritesAttribute; +import org.apache.nifi.annotation.documentation.WritesAttributes; import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.components.ValidationContext; import org.apache.nifi.components.ValidationResult; import org.apache.nifi.processor.ProcessContext; import org.apache.nifi.processor.ProcessorInitializationContext; -import org.apache.nifi.annotation.documentation.CapabilityDescription; -import org.apache.nifi.annotation.behavior.SideEffectFree; -import org.apache.nifi.annotation.documentation.Tags; import org.apache.nifi.processors.standard.util.FileTransfer; import org.apache.nifi.processors.standard.util.SFTPTransfer; @SideEffectFree @Tags({"sftp", "get", "retrieve", "files", "fetch", "remote", "ingest", "source", "input"}) @CapabilityDescription("Fetches files from an SFTP Server and creates FlowFiles from them") +@WritesAttributes({ @WritesAttribute(attribute = "filename", description = "The filename is set to the name of the file on the remote server"), + @WritesAttribute(attribute = "path", description = "The path is set to the path of the file's directory on the remote server. For example, if the property is set to /tmp, files picked up from /tmp will have the path attribute set to /tmp. If the 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"), + @WritesAttribute(attribute = "file.lastModifiedTime", description = "The date and time that the source file was last modified"), + @WritesAttribute(attribute = "file.owner", description = "The numeric owner id of the source file"), + @WritesAttribute(attribute = "file.group", description = "The numeric group id of the source file"), + @WritesAttribute(attribute = "file.permissions", description = "The read/write/execute permissions of the source file"), + @WritesAttribute(attribute = "absolute.path", description = "The full/absolute path from where a file was picked up. The current 'path' attribute is still populated, but may be a relative path")}) +@SeeAlso(PutSFTP.class) public class GetSFTP extends GetFileTransfer { private List properties; diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/HashAttribute.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/HashAttribute.java index e39ab26bf2..932729168a 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/HashAttribute.java +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/HashAttribute.java @@ -29,6 +29,14 @@ import java.util.concurrent.atomic.AtomicReference; import java.util.regex.Matcher; import java.util.regex.Pattern; +import org.apache.commons.codec.digest.DigestUtils; +import org.apache.commons.lang3.StringUtils; +import org.apache.nifi.annotation.behavior.EventDriven; +import org.apache.nifi.annotation.behavior.SideEffectFree; +import org.apache.nifi.annotation.behavior.SupportsBatching; +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.documentation.Tags; +import org.apache.nifi.annotation.documentation.WritesAttribute; import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.flowfile.FlowFile; import org.apache.nifi.logging.ProcessorLog; @@ -37,16 +45,8 @@ 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.behavior.EventDriven; -import org.apache.nifi.annotation.behavior.SideEffectFree; -import org.apache.nifi.annotation.behavior.SupportsBatching; -import org.apache.nifi.annotation.documentation.Tags; import org.apache.nifi.processor.util.StandardValidators; -import org.apache.commons.codec.digest.DigestUtils; -import org.apache.commons.lang3.StringUtils; - /** *

    * This processor identifies groups of user-specified flowfile attributes and @@ -114,6 +114,7 @@ import org.apache.commons.lang3.StringUtils; + "and the value of the property is a regular expression that, if matched by the attribute value, will cause 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 will be used.") +@WritesAttribute(attribute="", description="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 property.") public class HashAttribute extends AbstractProcessor { public static final PropertyDescriptor HASH_VALUE_ATTRIBUTE = new PropertyDescriptor.Builder() diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/HashContent.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/HashContent.java index 9f8a16c4d9..8daf615b78 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/HashContent.java +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/HashContent.java @@ -32,6 +32,7 @@ import org.apache.nifi.annotation.behavior.EventDriven; import org.apache.nifi.annotation.behavior.SupportsBatching; import org.apache.nifi.annotation.documentation.CapabilityDescription; import org.apache.nifi.annotation.documentation.Tags; +import org.apache.nifi.annotation.documentation.WritesAttribute; import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.flowfile.FlowFile; import org.apache.nifi.logging.ProcessorLog; @@ -51,6 +52,7 @@ import org.apache.nifi.util.ObjectHolder; @SupportsBatching @Tags({"hash", "content", "MD5", "SHA-1", "SHA-256"}) @CapabilityDescription("Calculates a hash value for the Content of a FlowFile and puts that hash value on the FlowFile as an attribute whose name is determined by the property") +@WritesAttribute(attribute="", description="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 property") public class HashContent extends AbstractProcessor { public static final PropertyDescriptor ATTRIBUTE_NAME = new PropertyDescriptor.Builder() diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/IdentifyMimeType.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/IdentifyMimeType.java index 68880e67af..4c560e854b 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/IdentifyMimeType.java +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/IdentifyMimeType.java @@ -23,6 +23,12 @@ import java.util.Collections; import java.util.HashSet; import java.util.Set; +import org.apache.nifi.annotation.behavior.EventDriven; +import org.apache.nifi.annotation.behavior.SideEffectFree; +import org.apache.nifi.annotation.behavior.SupportsBatching; +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.documentation.Tags; +import org.apache.nifi.annotation.documentation.WritesAttribute; import org.apache.nifi.flowfile.FlowFile; import org.apache.nifi.flowfile.attributes.CoreAttributes; import org.apache.nifi.logging.ProcessorLog; @@ -31,11 +37,6 @@ 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.behavior.EventDriven; -import org.apache.nifi.annotation.behavior.SideEffectFree; -import org.apache.nifi.annotation.behavior.SupportsBatching; -import org.apache.nifi.annotation.documentation.Tags; import org.apache.nifi.processor.io.InputStreamCallback; import org.apache.nifi.util.ObjectHolder; import org.apache.tika.config.TikaConfig; @@ -46,6 +47,7 @@ import org.apache.tika.mime.MediaType; import org.apache.tika.mime.MimeType; import org.apache.tika.mime.MimeTypeException; + /** *

    * Attempts to detect the MIME Type of a FlowFile by examining its contents. If @@ -71,6 +73,7 @@ import org.apache.tika.mime.MimeTypeException; + "an attribute with the name 'mime.type' is added with the value being the MIME Type. If the MIME Type cannot be determined, " + "the value will be set to 'application/octet-stream'. In addition, the attribute mime.extension will be set if a common file " + "extension for the MIME Type is known.") +@WritesAttribute(attribute="mime.type", description="This 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") public class IdentifyMimeType extends AbstractProcessor { public static final Relationship REL_SUCCESS = new Relationship.Builder().name("success").description("All FlowFiles are routed to success").build(); diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/InvokeHTTP.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/InvokeHTTP.java index 41ecd91378..9b98623410 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/InvokeHTTP.java +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/InvokeHTTP.java @@ -16,7 +16,7 @@ */ package org.apache.nifi.processors.standard; -import static org.apache.commons.lang3.StringUtils.*; +import static org.apache.commons.lang3.StringUtils.trimToEmpty; import java.io.BufferedInputStream; import java.io.BufferedOutputStream; @@ -48,6 +48,12 @@ import javax.net.ssl.HttpsURLConnection; import javax.net.ssl.SSLContext; import javax.net.ssl.SSLSession; +import org.apache.commons.lang3.StringUtils; +import org.apache.nifi.annotation.behavior.SupportsBatching; +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.documentation.Tags; +import org.apache.nifi.annotation.documentation.WritesAttribute; +import org.apache.nifi.annotation.documentation.WritesAttributes; import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.flowfile.FlowFile; import org.apache.nifi.logging.ProcessorLog; @@ -55,20 +61,22 @@ import org.apache.nifi.processor.AbstractProcessor; import org.apache.nifi.processor.ProcessContext; import org.apache.nifi.processor.ProcessSession; import org.apache.nifi.processor.Relationship; -import org.apache.nifi.annotation.documentation.CapabilityDescription; -import org.apache.nifi.annotation.behavior.SupportsBatching; -import org.apache.nifi.annotation.documentation.Tags; import org.apache.nifi.processor.exception.ProcessException; import org.apache.nifi.processor.util.StandardValidators; import org.apache.nifi.ssl.SSLContextService; import org.apache.nifi.ssl.SSLContextService.ClientAuth; -import org.apache.commons.lang3.StringUtils; import org.joda.time.format.DateTimeFormat; import org.joda.time.format.DateTimeFormatter; @SupportsBatching @Tags({"http", "https", "rest", "client"}) @CapabilityDescription("An HTTP client processor which converts FlowFile attributes to HTTP headers, with configurable HTTP method, url, etc.") +@WritesAttributes({ @WritesAttribute(attribute = "invokehttp.status.code", description = "The status code that is returned"), + @WritesAttribute(attribute = "invokehttp.status.message", description = "The status message that is returned"), + @WritesAttribute(attribute = "invokehttp.response.body", description = "The response body"), + @WritesAttribute(attribute = "invokehttp.request.url", description = "The request URL"), + @WritesAttribute(attribute = "invokehttp.tx.id", description = "The transaction ID that is returned after reading the response"), + @WritesAttribute(attribute = "invokehttp.remote.dn", description = "The DN of the remote server") }) public final class InvokeHTTP extends AbstractProcessor { //-- properties --// diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/MergeContent.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/MergeContent.java index a78bc074fb..187c50a19a 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/MergeContent.java +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/MergeContent.java @@ -40,7 +40,12 @@ import org.apache.commons.compress.archivers.tar.TarArchiveOutputStream; import org.apache.nifi.annotation.behavior.SideEffectFree; import org.apache.nifi.annotation.behavior.TriggerWhenEmpty; import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.documentation.ReadsAttribute; +import org.apache.nifi.annotation.documentation.ReadsAttributes; +import org.apache.nifi.annotation.documentation.SeeAlso; import org.apache.nifi.annotation.documentation.Tags; +import org.apache.nifi.annotation.documentation.WritesAttribute; +import org.apache.nifi.annotation.documentation.WritesAttributes; import org.apache.nifi.components.AllowableValue; import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.components.PropertyValue; @@ -70,6 +75,18 @@ import org.apache.nifi.util.ObjectHolder; @TriggerWhenEmpty @Tags({"merge", "content", "correlation", "tar", "zip", "stream", "concatenation", "archive", "flowfile-stream", "flowfile-stream-v3"}) @CapabilityDescription("Merges a Group of FlowFiles together 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 Group of FlowFiles will not be created from FlowFiles in different connections. This processor updates the mime.type attribute as appropriate.") +@ReadsAttributes({ @ReadsAttribute(attribute = "fragment.identifier", description = "Applicable only if the property is set to Defragment. All FlowFiles with the same value for this attribute will be bundled together"), + @ReadsAttribute(attribute = "fragment.index", description = "Applicable only if the 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"), + @ReadsAttribute(attribute = "fragment.count", description = "Applicable only if the 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"), + @ReadsAttribute(attribute = "segment.original.filename", description = "Applicable only if the 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"), + @ReadsAttribute(attribute = "tar.permissions", description = "Applicable only if the 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") }) +@WritesAttributes({ @WritesAttribute(attribute = "filename", description = "When 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
    "), + @WritesAttribute(attribute = "merge.count", description = "The number of FlowFiles that were merged into this bundle"), + @WritesAttribute(attribute = "merge.bin.age", description = "The 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") }) +@SeeAlso(SegmentContent.class) public class MergeContent extends BinFiles { // preferred attributes diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/MonitorActivity.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/MonitorActivity.java index c5fce3c6bb..1e74cb455d 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/MonitorActivity.java +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/MonitorActivity.java @@ -30,6 +30,13 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicLong; +import org.apache.nifi.annotation.behavior.SideEffectFree; +import org.apache.nifi.annotation.behavior.TriggerSerially; +import org.apache.nifi.annotation.behavior.TriggerWhenEmpty; +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.documentation.Tags; +import org.apache.nifi.annotation.documentation.WritesAttribute; +import org.apache.nifi.annotation.documentation.WritesAttributes; import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.flowfile.FlowFile; import org.apache.nifi.flowfile.attributes.CoreAttributes; @@ -39,11 +46,6 @@ 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.behavior.SideEffectFree; -import org.apache.nifi.annotation.documentation.Tags; -import org.apache.nifi.annotation.behavior.TriggerSerially; -import org.apache.nifi.annotation.behavior.TriggerWhenEmpty; import org.apache.nifi.processor.io.OutputStreamCallback; import org.apache.nifi.processor.util.StandardValidators; @@ -53,6 +55,9 @@ import org.apache.nifi.processor.util.StandardValidators; @Tags({"monitor", "flow", "active", "inactive", "activity", "detection"}) @CapabilityDescription("Monitors the flow for activity and sends out an indicator when the flow has not had any data for " + "some specified amount of time and again when the flow's activity is restored") +@WritesAttributes({ + @WritesAttribute(attribute = "inactivityStartMillis", description = "The time at which Inactivity began, in the form of milliseconds since Epoch"), + @WritesAttribute(attribute = "inactivityDurationMillis", description = "The number of milliseconds that the inactivity has spanned") }) public class MonitorActivity extends AbstractProcessor { public static final PropertyDescriptor THRESHOLD = new PropertyDescriptor.Builder() diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PostHTTP.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PostHTTP.java index f8a33bc8e7..b1017784e5 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PostHTTP.java +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PostHTTP.java @@ -80,6 +80,12 @@ import org.apache.http.impl.conn.PoolingHttpClientConnectionManager; import org.apache.http.protocol.HttpContext; import org.apache.http.protocol.HttpCoreContext; import org.apache.http.util.EntityUtils; +import org.apache.nifi.annotation.behavior.SupportsBatching; +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.documentation.ReadsAttribute; +import org.apache.nifi.annotation.documentation.Tags; +import org.apache.nifi.annotation.lifecycle.OnScheduled; +import org.apache.nifi.annotation.lifecycle.OnStopped; import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.components.ValidationContext; import org.apache.nifi.components.ValidationResult; @@ -92,11 +98,6 @@ 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.lifecycle.OnScheduled; -import org.apache.nifi.annotation.lifecycle.OnStopped; -import org.apache.nifi.annotation.behavior.SupportsBatching; -import org.apache.nifi.annotation.documentation.Tags; import org.apache.nifi.processor.exception.ProcessException; import org.apache.nifi.processor.io.InputStreamCallback; import org.apache.nifi.processor.util.StandardValidators; @@ -120,6 +121,7 @@ import com.sun.jersey.api.client.ClientResponse.Status; @SupportsBatching @Tags({"http", "https", "remote", "copy", "archive"}) @CapabilityDescription("Performs an HTTP Post with the content of the FlowFile") +@ReadsAttribute(attribute="mime.type", description="If not sending data as a FlowFile, the mime.type attribute will be used to set the HTTP Header for Content-Type") public class PostHTTP extends AbstractProcessor { public static final String CONTENT_TYPE = "Content-Type"; diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PutFTP.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PutFTP.java index 458fcbc30b..6d80038e50 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PutFTP.java +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PutFTP.java @@ -26,20 +26,22 @@ import java.util.concurrent.atomic.AtomicReference; import java.util.regex.Matcher; import java.util.regex.Pattern; +import org.apache.nifi.annotation.behavior.SupportsBatching; +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.documentation.SeeAlso; +import org.apache.nifi.annotation.documentation.Tags; +import org.apache.nifi.annotation.lifecycle.OnScheduled; import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.flowfile.FlowFile; import org.apache.nifi.processor.ProcessContext; import org.apache.nifi.processor.ProcessorInitializationContext; -import org.apache.nifi.annotation.documentation.CapabilityDescription; -import org.apache.nifi.annotation.lifecycle.OnScheduled; -import org.apache.nifi.annotation.behavior.SupportsBatching; -import org.apache.nifi.annotation.documentation.Tags; import org.apache.nifi.processor.util.StandardValidators; import org.apache.nifi.processors.standard.util.FTPTransfer; @SupportsBatching @Tags({"remote", "copy", "egress", "put", "ftp", "archive", "files"}) @CapabilityDescription("Sends FlowFiles to an FTP Server") +@SeeAlso(GetFTP.class) public class PutFTP extends PutFileTransfer { private static final Pattern PRE_SEND_CMD_PATTERN = Pattern.compile("^pre\\.cmd\\.(\\d+)$"); diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PutFile.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PutFile.java index 70ac5acb1a..ba78133a0f 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PutFile.java +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PutFile.java @@ -34,6 +34,10 @@ import java.util.Set; import java.util.concurrent.TimeUnit; import java.util.regex.Pattern; +import org.apache.nifi.annotation.behavior.SupportsBatching; +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.documentation.SeeAlso; +import org.apache.nifi.annotation.documentation.Tags; import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.flowfile.FlowFile; import org.apache.nifi.flowfile.attributes.CoreAttributes; @@ -43,9 +47,6 @@ 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.behavior.SupportsBatching; -import org.apache.nifi.annotation.documentation.Tags; import org.apache.nifi.processor.exception.ProcessException; import org.apache.nifi.processor.util.StandardValidators; import org.apache.nifi.util.StopWatch; @@ -53,6 +54,7 @@ import org.apache.nifi.util.StopWatch; @SupportsBatching @Tags({"put", "local", "copy", "archive", "files", "filesystem"}) @CapabilityDescription("Writes the contents of a FlowFile to the local file system") +@SeeAlso(GetFile.class) public class PutFile extends AbstractProcessor { public static final String REPLACE_RESOLUTION = "replace"; diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PutJMS.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PutJMS.java index 99c7bb7c93..c7c78a139a 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PutJMS.java +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PutJMS.java @@ -28,6 +28,7 @@ import static org.apache.nifi.processors.standard.util.JmsFactory.PROP_TYPE_OBJE import static org.apache.nifi.processors.standard.util.JmsFactory.PROP_TYPE_SHORT; import static org.apache.nifi.processors.standard.util.JmsFactory.PROP_TYPE_STRING; import static org.apache.nifi.processors.standard.util.JmsProperties.ATTRIBUTES_TO_JMS_PROPS; +import static org.apache.nifi.processors.standard.util.JmsProperties.BATCH_SIZE; import static org.apache.nifi.processors.standard.util.JmsProperties.CLIENT_ID_PREFIX; import static org.apache.nifi.processors.standard.util.JmsProperties.DESTINATION_NAME; import static org.apache.nifi.processors.standard.util.JmsProperties.DESTINATION_TYPE; @@ -44,7 +45,6 @@ import static org.apache.nifi.processors.standard.util.JmsProperties.MSG_TYPE_MA import static org.apache.nifi.processors.standard.util.JmsProperties.PASSWORD; import static org.apache.nifi.processors.standard.util.JmsProperties.REPLY_TO_QUEUE; import static org.apache.nifi.processors.standard.util.JmsProperties.TIMEOUT; -import static org.apache.nifi.processors.standard.util.JmsProperties.BATCH_SIZE; import static org.apache.nifi.processors.standard.util.JmsProperties.URL; import static org.apache.nifi.processors.standard.util.JmsProperties.USERNAME; @@ -71,11 +71,11 @@ import javax.jms.Session; import javax.jms.StreamMessage; import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.documentation.SeeAlso; import org.apache.nifi.annotation.documentation.Tags; import org.apache.nifi.annotation.lifecycle.OnStopped; import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.flowfile.FlowFile; -import org.apache.nifi.stream.io.StreamUtils; import org.apache.nifi.logging.ProcessorLog; import org.apache.nifi.processor.AbstractProcessor; import org.apache.nifi.processor.DataUnit; @@ -86,9 +86,11 @@ import org.apache.nifi.processor.exception.ProcessException; import org.apache.nifi.processor.io.InputStreamCallback; import org.apache.nifi.processors.standard.util.JmsFactory; import org.apache.nifi.processors.standard.util.WrappedMessageProducer; +import org.apache.nifi.stream.io.StreamUtils; @Tags({"jms", "send", "put"}) @CapabilityDescription("Creates a JMS Message from the contents of a FlowFile and sends the message to a JMS Server") +@SeeAlso({GetJMSQueue.class, GetJMSTopic.class}) public class PutJMS extends AbstractProcessor { public static final Charset UTF8 = Charset.forName("UTF-8"); diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PutSFTP.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PutSFTP.java index cfd522c703..d3b93e7e87 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PutSFTP.java +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PutSFTP.java @@ -20,17 +20,19 @@ import java.util.ArrayList; import java.util.Collections; import java.util.List; +import org.apache.nifi.annotation.behavior.SupportsBatching; +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.documentation.SeeAlso; +import org.apache.nifi.annotation.documentation.Tags; import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.processor.ProcessContext; import org.apache.nifi.processor.ProcessorInitializationContext; -import org.apache.nifi.annotation.documentation.CapabilityDescription; -import org.apache.nifi.annotation.behavior.SupportsBatching; -import org.apache.nifi.annotation.documentation.Tags; import org.apache.nifi.processors.standard.util.SFTPTransfer; @SupportsBatching @Tags({"remote", "copy", "egress", "put", "sftp", "archive", "files"}) @CapabilityDescription("Sends FlowFiles to an SFTP Server") +@SeeAlso(GetSFTP.class) public class PutSFTP extends PutFileTransfer { private List properties; diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ScanContent.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ScanContent.java index d9f20349d1..55890e2b7b 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ScanContent.java +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ScanContent.java @@ -34,26 +34,27 @@ import java.util.Set; import java.util.concurrent.atomic.AtomicReference; import java.util.concurrent.locks.ReentrantLock; +import org.apache.nifi.annotation.behavior.EventDriven; +import org.apache.nifi.annotation.behavior.SideEffectFree; +import org.apache.nifi.annotation.behavior.SupportsBatching; +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.documentation.Tags; +import org.apache.nifi.annotation.documentation.WritesAttribute; import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.flowfile.FlowFile; -import org.apache.nifi.stream.io.BufferedInputStream; -import org.apache.nifi.util.file.monitor.LastModifiedMonitor; -import org.apache.nifi.util.file.monitor.SynchronousFileWatcher; import org.apache.nifi.logging.ProcessorLog; 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.behavior.EventDriven; -import org.apache.nifi.annotation.behavior.SideEffectFree; -import org.apache.nifi.annotation.behavior.SupportsBatching; -import org.apache.nifi.annotation.documentation.Tags; import org.apache.nifi.processor.exception.ProcessException; import org.apache.nifi.processor.io.InputStreamCallback; import org.apache.nifi.processor.util.StandardValidators; +import org.apache.nifi.stream.io.BufferedInputStream; import org.apache.nifi.util.ObjectHolder; +import org.apache.nifi.util.file.monitor.LastModifiedMonitor; +import org.apache.nifi.util.file.monitor.SynchronousFileWatcher; import org.apache.nifi.util.search.Search; import org.apache.nifi.util.search.SearchTerm; import org.apache.nifi.util.search.ahocorasick.AhoCorasick; @@ -64,6 +65,7 @@ import org.apache.nifi.util.search.ahocorasick.SearchState; @SupportsBatching @Tags({"aho-corasick", "scan", "content", "byte sequence", "search", "find", "dictionary"}) @CapabilityDescription("Scans the content of FlowFiles for terms that are found in a user-supplied dictionary. If a term is matched, the UTF-8 encoded version of the term will be added to the FlowFile using the 'matching.term' attribute") +@WritesAttribute(attribute="matching.term", description="The 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") public class ScanContent extends AbstractProcessor { public static final String TEXT_ENCODING = "text"; diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/SegmentContent.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/SegmentContent.java index cf0539ed71..f2ce9a1e61 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/SegmentContent.java +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/SegmentContent.java @@ -29,7 +29,10 @@ import org.apache.nifi.annotation.behavior.EventDriven; import org.apache.nifi.annotation.behavior.SideEffectFree; import org.apache.nifi.annotation.behavior.SupportsBatching; import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.documentation.SeeAlso; import org.apache.nifi.annotation.documentation.Tags; +import org.apache.nifi.annotation.documentation.WritesAttribute; +import org.apache.nifi.annotation.documentation.WritesAttributes; import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.flowfile.FlowFile; import org.apache.nifi.flowfile.attributes.CoreAttributes; @@ -48,6 +51,15 @@ import org.apache.nifi.processor.util.StandardValidators; @CapabilityDescription("Segments a FlowFile into multiple smaller segments on byte boundaries. Each segment is given the following attributes: " + "fragment.identifier, fragment.index, fragment.count, segment.original.filename; these attributes can then be used by the " + "MergeContent processor in order to reconstitute the original FlowFile") +@WritesAttributes({ @WritesAttribute(attribute = "segment.identifier", description = "All segments produced from the same parent FlowFile will have the same randomly generated UUID added for this attribute. This attribute is added to maintain backward compatibility, but the fragment.identifier is preferred, as it is designed to work in conjunction with the MergeContent Processor"), + @WritesAttribute(attribute = "segment.index", description = "A one-up number that indicates the ordering of the segments that were created from a single parent FlowFile. This attribute is added to maintain backward compatibility, but the fragment.index is preferred, as it is designed to work in conjunction with the MergeContent Processor"), + @WritesAttribute(attribute = "segment.count", description = "The number of segments generated from the parent FlowFile. This attribute is added to maintain backward compatibility, but the fragment.count is preferred, as it is designed to work in conjunction with the MergeContent Processor"), + @WritesAttribute(attribute = "fragment.identifier", description = "All segments produced from the same parent FlowFile will have the same randomly generated UUID added for this attribute"), + @WritesAttribute(attribute = "fragment.index", description = "A one-up number that indicates the ordering of the segments that were created from a single parent FlowFile"), + @WritesAttribute(attribute = "fragment.count", description = "The number of segments generated from the parent FlowFile"), + @WritesAttribute(attribute = "segment.original.filename ", description = "The filename of the parent FlowFile"), + @WritesAttribute(attribute = "segment.original.filename ", description = "The filename will be updated to include the parent's filename, the segment index, and the segment count") }) +@SeeAlso(MergeContent.class) public class SegmentContent extends AbstractProcessor { public static final String SEGMENT_ID = "segment.identifier"; diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/SplitContent.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/SplitContent.java index 8c00a7e650..1c19805677 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/SplitContent.java +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/SplitContent.java @@ -28,35 +28,42 @@ import java.util.Set; import java.util.UUID; import java.util.concurrent.atomic.AtomicReference; +import org.apache.commons.codec.binary.Hex; +import org.apache.nifi.annotation.behavior.EventDriven; +import org.apache.nifi.annotation.behavior.SideEffectFree; +import org.apache.nifi.annotation.behavior.SupportsBatching; +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.documentation.SeeAlso; +import org.apache.nifi.annotation.documentation.Tags; +import org.apache.nifi.annotation.documentation.WritesAttribute; +import org.apache.nifi.annotation.documentation.WritesAttributes; import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.components.ValidationContext; import org.apache.nifi.components.ValidationResult; import org.apache.nifi.components.Validator; import org.apache.nifi.flowfile.FlowFile; import org.apache.nifi.flowfile.attributes.CoreAttributes; -import org.apache.nifi.stream.io.BufferedInputStream; import org.apache.nifi.logging.ProcessorLog; 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.behavior.EventDriven; -import org.apache.nifi.annotation.behavior.SideEffectFree; -import org.apache.nifi.annotation.behavior.SupportsBatching; -import org.apache.nifi.annotation.documentation.Tags; import org.apache.nifi.processor.io.InputStreamCallback; +import org.apache.nifi.stream.io.BufferedInputStream; import org.apache.nifi.util.NaiveSearchRingBuffer; import org.apache.nifi.util.Tuple; -import org.apache.commons.codec.binary.Hex; - @EventDriven @SideEffectFree @SupportsBatching @Tags({"content", "split", "binary"}) @CapabilityDescription("Splits incoming FlowFiles by a specified byte sequence") +@WritesAttributes({ @WritesAttribute(attribute = "fragment.identifier", description = "All split FlowFiles produced from the same parent FlowFile will have the same randomly generated UUID added for this attribute"), + @WritesAttribute(attribute = "fragment.index", description = "A one-up number that indicates the ordering of the split FlowFiles that were created from a single parent FlowFile"), + @WritesAttribute(attribute = "fragment.count", description = "The number of split FlowFiles generated from the parent FlowFile"), + @WritesAttribute(attribute = "segment.original.filename ", description = "The filename of the parent FlowFile") }) +@SeeAlso(MergeContent.class) public class SplitContent extends AbstractProcessor { // attribute keys diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/SplitText.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/SplitText.java index 2a81b64754..da201887d2 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/SplitText.java +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/SplitText.java @@ -30,6 +30,9 @@ 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.SeeAlso; +import org.apache.nifi.annotation.documentation.WritesAttribute; +import org.apache.nifi.annotation.documentation.WritesAttributes; import org.apache.nifi.annotation.behavior.EventDriven; import org.apache.nifi.annotation.behavior.SideEffectFree; import org.apache.nifi.annotation.behavior.SupportsBatching; @@ -58,6 +61,13 @@ import java.util.UUID; @SupportsBatching @Tags({"split", "text"}) @CapabilityDescription("Splits a text file into multiple smaller text files on line boundaries, each having up to a configured number of lines") +@WritesAttributes({ + @WritesAttribute(attribute = "text.line.count", description = "The number of lines of text from the original FlowFile that were copied to this FlowFile"), + @WritesAttribute(attribute = "fragment.identifier", description = "All split FlowFiles produced from the same parent FlowFile will have the same randomly generated UUID added for this attribute"), + @WritesAttribute(attribute = "fragment.index", description = "A one-up number that indicates the ordering of the split FlowFiles that were created from a single parent FlowFile"), + @WritesAttribute(attribute = "fragment.count", description = "The number of split FlowFiles generated from the parent FlowFile"), + @WritesAttribute(attribute = "segment.original.filename ", description = "The filename of the parent FlowFile") }) +@SeeAlso(MergeContent.class) public class SplitText extends AbstractProcessor { // attribute keys diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/UnpackContent.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/UnpackContent.java index b30e780b72..637b0c10a0 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/UnpackContent.java +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/UnpackContent.java @@ -30,42 +30,53 @@ import java.util.Map; import java.util.Set; import java.util.UUID; +import org.apache.commons.compress.archivers.ArchiveEntry; +import org.apache.commons.compress.archivers.tar.TarArchiveEntry; +import org.apache.commons.compress.archivers.tar.TarArchiveInputStream; +import org.apache.commons.compress.archivers.zip.ZipArchiveInputStream; +import org.apache.nifi.annotation.behavior.EventDriven; +import org.apache.nifi.annotation.behavior.SideEffectFree; +import org.apache.nifi.annotation.behavior.SupportsBatching; +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.documentation.ReadsAttribute; +import org.apache.nifi.annotation.documentation.SeeAlso; +import org.apache.nifi.annotation.documentation.Tags; +import org.apache.nifi.annotation.documentation.WritesAttribute; +import org.apache.nifi.annotation.documentation.WritesAttributes; import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.flowfile.FlowFile; import org.apache.nifi.flowfile.attributes.CoreAttributes; -import org.apache.nifi.stream.io.BufferedInputStream; -import org.apache.nifi.stream.io.BufferedOutputStream; -import org.apache.nifi.stream.io.StreamUtils; import org.apache.nifi.logging.ProcessorLog; 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.behavior.EventDriven; -import org.apache.nifi.annotation.behavior.SideEffectFree; -import org.apache.nifi.annotation.behavior.SupportsBatching; -import org.apache.nifi.annotation.documentation.Tags; import org.apache.nifi.processor.exception.ProcessException; import org.apache.nifi.processor.io.InputStreamCallback; import org.apache.nifi.processor.io.OutputStreamCallback; +import org.apache.nifi.stream.io.BufferedInputStream; +import org.apache.nifi.stream.io.BufferedOutputStream; +import org.apache.nifi.stream.io.StreamUtils; import org.apache.nifi.util.FlowFileUnpackager; import org.apache.nifi.util.FlowFileUnpackagerV1; import org.apache.nifi.util.FlowFileUnpackagerV2; import org.apache.nifi.util.FlowFileUnpackagerV3; import org.apache.nifi.util.ObjectHolder; -import org.apache.commons.compress.archivers.ArchiveEntry; -import org.apache.commons.compress.archivers.tar.TarArchiveEntry; -import org.apache.commons.compress.archivers.tar.TarArchiveInputStream; -import org.apache.commons.compress.archivers.zip.ZipArchiveInputStream; - @EventDriven @SideEffectFree @SupportsBatching @Tags({"Unpack", "un-merge", "tar", "zip", "archive", "flowfile-stream", "flowfile-stream-v3"}) @CapabilityDescription("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") +@ReadsAttribute(attribute = "mime.type", description = "If the property is set to use mime.type attribute, this attribute is used to determine the FlowFile's MIME Type. In this case, if the attribute is set to application/tar, the TAR Packaging Format will be used. If the attribute is set to application/zip, the ZIP Packaging Format will be used. If the attribute is set to application/flowfile-v3 or application/flowfile-v2 or application/flowfile-v1, the appropriate FlowFile Packaging Format will be used. If this attribute is missing, the FlowFile will be routed to 'failure'. Otherwise, if the attribute's value is not one of those mentioned above, the FlowFile will be routed to 'success' without being unpacked") +@WritesAttributes({ + @WritesAttribute(attribute = "mime.type", description = "If the FlowFile is successfully unpacked, its MIME Type is no longer known, so the mime.type attribute is set to application/octet-stream."), + @WritesAttribute(attribute = "fragment.identifier", description = "All unpacked FlowFiles produced from the same parent FlowFile will have the same randomly generated UUID added for this attribute"), + @WritesAttribute(attribute = "fragment.index", description = "A one-up number that indicates the ordering of the unpacked FlowFiles that were created from a single parent FlowFile"), + @WritesAttribute(attribute = "fragment.count", description = "The number of unpacked FlowFiles generated from the parent FlowFile"), + @WritesAttribute(attribute = "segment.original.filename ", description = "The filename of the parent FlowFile. Extensions of .tar, .zip or .pkg are removed because the MergeContent processor automatically adds those extensions if it is used to rebuild the original FlowFile") }) +@SeeAlso(MergeContent.class) public class UnpackContent extends AbstractProcessor { public static final String AUTO_DETECT_FORMAT = "use mime.type attribute"; From 6722fc9dbc2fcbe91ca69e12a5d85cd3134577de Mon Sep 17 00:00:00 2001 From: danbress Date: Thu, 5 Mar 2015 14:01:24 -0500 Subject: [PATCH 042/116] NIFI-309 better support for See also using class names --- .../org/apache/nifi/annotation/documentation/SeeAlso.java | 4 ++-- .../nifi/documentation/html/HtmlDocumentationWriter.java | 4 ++-- .../documentation/html/HtmlProcessorDocumentationWriter.java | 3 +++ .../nifi/documentation/example/FullyDocumentedProcessor.java | 2 +- .../org/apache/nifi/processors/standard/DetectDuplicate.java | 2 +- 5 files changed, 9 insertions(+), 6 deletions(-) diff --git a/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/documentation/SeeAlso.java b/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/documentation/SeeAlso.java index 3ad8081984..d664117984 100644 --- a/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/documentation/SeeAlso.java +++ b/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/documentation/SeeAlso.java @@ -46,7 +46,7 @@ public @interface SeeAlso { * * @return */ - public Class[] value(); + public Class[] value() default {}; /** * Fully qualified class names you want to link to. Use this when the class @@ -55,5 +55,5 @@ public @interface SeeAlso { * * @return */ - public String[] classNames() default ""; + public String[] classNames() default {}; } 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 1bfdd2709f..f3c0edf742 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 @@ -144,7 +144,7 @@ public class HtmlDocumentationWriter implements DocumentationWriter { private void writeSeeAlso(ConfigurableComponent configurableComponent, XMLStreamWriter xmlStreamWriter) throws XMLStreamException { final SeeAlso seeAlso = configurableComponent.getClass().getAnnotation(SeeAlso.class); - if (seeAlso != null && seeAlso.value().length > 0) { + if (seeAlso != null) { writeSimpleElement(xmlStreamWriter, "h3", "See Also:"); xmlStreamWriter.writeStartElement("p"); int index = 0; @@ -167,7 +167,7 @@ public class HtmlDocumentationWriter implements DocumentationWriter { final String link = "../" + linkedComponent + "/index.html"; - final int indexOfLastPeriod = Math.min(0, linkedComponent.lastIndexOf(".")); + final int indexOfLastPeriod = Math.max(0, linkedComponent.lastIndexOf(".")); writeLink(xmlStreamWriter, linkedComponent.substring(indexOfLastPeriod), link); 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 f83b2f69d9..af218d879d 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 @@ -88,9 +88,11 @@ public class HtmlProcessorDocumentationWriter extends HtmlDocumentationWriter { xmlStreamWriter.writeStartElement("tr"); writeSimpleElement(xmlStreamWriter, "td", StringUtils.defaultIfBlank(attribute.attribute(), "Not Specified")); + // TODO allow for HTML characters here. writeSimpleElement(xmlStreamWriter, "td", StringUtils.defaultIfBlank(attribute.description(), "Not Specified")); xmlStreamWriter.writeEndElement(); + } xmlStreamWriter.writeEndElement(); @@ -123,6 +125,7 @@ public class HtmlProcessorDocumentationWriter extends HtmlDocumentationWriter { xmlStreamWriter.writeStartElement("tr"); writeSimpleElement(xmlStreamWriter, "td", StringUtils.defaultIfBlank(attribute.attribute(), "Not Specified")); + // TODO allow for HTML characters here. writeSimpleElement(xmlStreamWriter, "td", StringUtils.defaultIfBlank(attribute.description(), "Not Specified")); xmlStreamWriter.writeEndElement(); 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 b9bce9123a..84dc88f5e6 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 @@ -42,7 +42,7 @@ import org.apache.nifi.processor.util.StandardValidators; @CapabilityDescription("This is a processor that is used to test documentation.") @WritesAttributes({@WritesAttribute(attribute="first", description="this is the first attribute i write"), @WritesAttribute(attribute="second")}) @ReadsAttribute(attribute = "incoming", description="this specifies the format of the thing") -@SeeAlso({FullyDocumentedControllerService.class, FullyDocumentedReportingTask.class}) +@SeeAlso(value={FullyDocumentedControllerService.class, FullyDocumentedReportingTask.class}, classNames={"org.apache.nifi.processor.ExampleProcessor"}) public class FullyDocumentedProcessor extends AbstractProcessor { public static final PropertyDescriptor DIRECTORY = new PropertyDescriptor.Builder().name("Input Directory") diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/DetectDuplicate.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/DetectDuplicate.java index f2a9e3ed56..eb0d6c7f67 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/DetectDuplicate.java +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/DetectDuplicate.java @@ -57,7 +57,7 @@ import org.apache.nifi.processor.util.StandardValidators; + "\"description\", which is specified in the property. If the FlowFile is not determined to be a duplicate, the Processor " + "routes the FlowFile to 'non-duplicate'") @WritesAttribute(attribute="original.flowfile.description", description="All 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.") -@SeeAlso(value={DistributedMapCacheClient.class}, classNames={"org.apache.nifi.distributed.cache.server.map.DistributedMapCacheServer"}) +@SeeAlso(classNames={"org.apache.nifi.distributed.cache.client.DistributedMapCacheClientService", "org.apache.nifi.distributed.cache.server.map.DistributedMapCacheServer"}) public class DetectDuplicate extends AbstractProcessor { public static final String ORIGINAL_DESCRIPTION_ATTRIBUTE_NAME = "original.flowfile.description"; From 84f5a08c04808cfddc047432850b9874a3f7a230 Mon Sep 17 00:00:00 2001 From: danbress Date: Thu, 5 Mar 2015 14:06:32 -0500 Subject: [PATCH 043/116] NIFI-309 adding SeeAlso to standard services --- .../cache/client/DistributedMapCacheClientService.java | 2 ++ .../cache/client/DistributedSetCacheClientService.java | 2 ++ .../distributed/cache/server/map/DistributedMapCacheServer.java | 2 ++ 3 files changed, 6 insertions(+) diff --git a/nifi/nifi-nar-bundles/nifi-standard-services/nifi-distributed-cache-services-bundle/nifi-distributed-cache-client-service/src/main/java/org/apache/nifi/distributed/cache/client/DistributedMapCacheClientService.java b/nifi/nifi-nar-bundles/nifi-standard-services/nifi-distributed-cache-services-bundle/nifi-distributed-cache-client-service/src/main/java/org/apache/nifi/distributed/cache/client/DistributedMapCacheClientService.java index 6dad80b9b4..feb6199029 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-services/nifi-distributed-cache-services-bundle/nifi-distributed-cache-client-service/src/main/java/org/apache/nifi/distributed/cache/client/DistributedMapCacheClientService.java +++ b/nifi/nifi-nar-bundles/nifi-standard-services/nifi-distributed-cache-services-bundle/nifi-distributed-cache-client-service/src/main/java/org/apache/nifi/distributed/cache/client/DistributedMapCacheClientService.java @@ -24,6 +24,7 @@ import java.util.concurrent.BlockingQueue; import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.TimeUnit; +import org.apache.nifi.annotation.documentation.SeeAlso; import org.apache.nifi.annotation.lifecycle.OnEnabled; import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.controller.AbstractControllerService; @@ -40,6 +41,7 @@ import org.apache.nifi.stream.io.DataOutputStream; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +@SeeAlso(classNames={"org.apache.nifi.distributed.cache.server.map.DistributedMapCacheServer", "org.apache.nifi.ssl.StandardSSLContextService"}) public class DistributedMapCacheClientService extends AbstractControllerService implements DistributedMapCacheClient { private static final Logger logger = LoggerFactory.getLogger(DistributedMapCacheClientService.class); diff --git a/nifi/nifi-nar-bundles/nifi-standard-services/nifi-distributed-cache-services-bundle/nifi-distributed-cache-client-service/src/main/java/org/apache/nifi/distributed/cache/client/DistributedSetCacheClientService.java b/nifi/nifi-nar-bundles/nifi-standard-services/nifi-distributed-cache-services-bundle/nifi-distributed-cache-client-service/src/main/java/org/apache/nifi/distributed/cache/client/DistributedSetCacheClientService.java index dcc55589fd..7f383945e9 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-services/nifi-distributed-cache-services-bundle/nifi-distributed-cache-client-service/src/main/java/org/apache/nifi/distributed/cache/client/DistributedSetCacheClientService.java +++ b/nifi/nifi-nar-bundles/nifi-standard-services/nifi-distributed-cache-services-bundle/nifi-distributed-cache-client-service/src/main/java/org/apache/nifi/distributed/cache/client/DistributedSetCacheClientService.java @@ -24,6 +24,7 @@ import java.util.concurrent.BlockingQueue; import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.TimeUnit; +import org.apache.nifi.annotation.documentation.SeeAlso; import org.apache.nifi.annotation.lifecycle.OnEnabled; import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.controller.AbstractControllerService; @@ -40,6 +41,7 @@ import org.apache.nifi.stream.io.DataOutputStream; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +@SeeAlso(classNames={"org.apache.nifi.distributed.cache.server.map.DistributedMapCacheServer", "org.apache.nifi.ssl.StandardSSLContextService"}) public class DistributedSetCacheClientService extends AbstractControllerService implements DistributedSetCacheClient { private static final Logger logger = LoggerFactory.getLogger(DistributedMapCacheClientService.class); diff --git a/nifi/nifi-nar-bundles/nifi-standard-services/nifi-distributed-cache-services-bundle/nifi-distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/map/DistributedMapCacheServer.java b/nifi/nifi-nar-bundles/nifi-standard-services/nifi-distributed-cache-services-bundle/nifi-distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/map/DistributedMapCacheServer.java index 6c62849149..0594dd4b27 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-services/nifi-distributed-cache-services-bundle/nifi-distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/map/DistributedMapCacheServer.java +++ b/nifi/nifi-nar-bundles/nifi-standard-services/nifi-distributed-cache-services-bundle/nifi-distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/map/DistributedMapCacheServer.java @@ -21,6 +21,7 @@ import java.io.File; import javax.net.ssl.SSLContext; import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.documentation.SeeAlso; import org.apache.nifi.annotation.documentation.Tags; import org.apache.nifi.controller.ConfigurationContext; import org.apache.nifi.distributed.cache.server.CacheServer; @@ -32,6 +33,7 @@ import org.apache.nifi.ssl.SSLContextService.ClientAuth; @Tags({"distributed", "cluster", "map", "cache", "server", "key/value"}) @CapabilityDescription("Provides a map (key/value) cache that can be accessed over a socket. Interaction with this service" + " is typically accomplished via a DistributedMapCacheClient service.") +@SeeAlso(classNames={"org.apache.nifi.distributed.cache.client.DistributedMapCacheClientService", "org.apache.nifi.ssl.StandardSSLContextService"}) public class DistributedMapCacheServer extends DistributedCacheServer { @Override From 458540e5d2e94af630c5bdeb21976171483c9887 Mon Sep 17 00:00:00 2001 From: danbress Date: Thu, 5 Mar 2015 14:19:20 -0500 Subject: [PATCH 044/116] NIFI-309 adding documentation attributes to processors --- .../hadoop/CreateHadoopSequenceFile.java | 2 ++ .../nifi/processors/hadoop/GetHDFS.java | 21 ++++++++++++------- .../hadoop/GetHDFSSequenceFile.java | 2 ++ .../nifi/processors/hadoop/PutHDFS.java | 20 +++++++++++------- .../nifi/processors/kafka/GetKafka.java | 6 ++++++ 5 files changed, 36 insertions(+), 15 deletions(-) diff --git a/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/CreateHadoopSequenceFile.java b/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/CreateHadoopSequenceFile.java index a031923bcd..98c1f1ce74 100644 --- a/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/CreateHadoopSequenceFile.java +++ b/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/CreateHadoopSequenceFile.java @@ -25,6 +25,7 @@ import java.util.Set; import org.apache.hadoop.io.SequenceFile.CompressionType; import org.apache.nifi.annotation.behavior.SideEffectFree; import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.documentation.SeeAlso; import org.apache.nifi.annotation.documentation.Tags; import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.flowfile.FlowFile; @@ -67,6 +68,7 @@ import org.apache.nifi.processors.hadoop.util.SequenceFileWriter; @SideEffectFree @Tags({"hadoop", "sequence file", "create", "sequencefile"}) @CapabilityDescription("Creates Hadoop Sequence Files from incoming flow files") +@SeeAlso(PutHDFS.class) public class CreateHadoopSequenceFile extends AbstractHadoopProcessor { public static final String TAR_FORMAT = "tar"; diff --git a/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/GetHDFS.java b/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/GetHDFS.java index 0610d8f313..12a6c9fe30 100644 --- a/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/GetHDFS.java +++ b/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/GetHDFS.java @@ -31,9 +31,19 @@ import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.ReentrantLock; import java.util.regex.Pattern; +import org.apache.commons.io.IOUtils; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.PathFilter; import org.apache.nifi.annotation.behavior.TriggerWhenEmpty; import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.documentation.SeeAlso; import org.apache.nifi.annotation.documentation.Tags; +import org.apache.nifi.annotation.documentation.WritesAttribute; +import org.apache.nifi.annotation.documentation.WritesAttributes; import org.apache.nifi.annotation.lifecycle.OnScheduled; import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.components.ValidationContext; @@ -47,13 +57,6 @@ import org.apache.nifi.processor.Relationship; import org.apache.nifi.processor.exception.ProcessException; import org.apache.nifi.processor.util.StandardValidators; import org.apache.nifi.util.StopWatch; -import org.apache.commons.io.IOUtils; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FSDataInputStream; -import org.apache.hadoop.fs.FileStatus; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.fs.PathFilter; /** * This processor reads files from HDFS into NiFi FlowFiles. @@ -61,6 +64,10 @@ import org.apache.hadoop.fs.PathFilter; @TriggerWhenEmpty @Tags({"hadoop", "HDFS", "get", "fetch", "ingest", "source", "filesystem"}) @CapabilityDescription("Fetch files from Hadoop Distributed File System (HDFS) into FlowFiles") +@WritesAttributes({ + @WritesAttribute(attribute = "filename", description = "The name of the file that was read from HDFS."), + @WritesAttribute(attribute = "path", description = "The 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\".") }) +@SeeAlso(PutHDFS.class) public class GetHDFS extends AbstractHadoopProcessor { public static final String BUFFER_SIZE_KEY = "io.file.buffer.size"; diff --git a/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/GetHDFSSequenceFile.java b/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/GetHDFSSequenceFile.java index ec8b5e67d4..88e725b19f 100644 --- a/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/GetHDFSSequenceFile.java +++ b/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/GetHDFSSequenceFile.java @@ -24,6 +24,7 @@ import java.util.concurrent.TimeUnit; import org.apache.nifi.annotation.behavior.TriggerWhenEmpty; import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.documentation.SeeAlso; import org.apache.nifi.annotation.documentation.Tags; import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.flowfile.FlowFile; @@ -57,6 +58,7 @@ import org.apache.hadoop.fs.Path; @TriggerWhenEmpty @Tags({"hadoop", "HDFS", "get", "fetch", "ingest", "source", "sequence file"}) @CapabilityDescription("Fetch sequence files from Hadoop Distributed File System (HDFS) into FlowFiles") +@SeeAlso(PutHDFS.class) public class GetHDFSSequenceFile extends GetHDFS { static final String VALUE_ONLY = "VALUE ONLY"; diff --git a/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/PutHDFS.java b/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/PutHDFS.java index 9a5aa7475b..cb166c9a91 100644 --- a/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/PutHDFS.java +++ b/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/PutHDFS.java @@ -26,8 +26,16 @@ import java.util.List; import java.util.Set; import java.util.concurrent.TimeUnit; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.permission.FsPermission; +import org.apache.hadoop.ipc.RemoteException; import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.documentation.SeeAlso; import org.apache.nifi.annotation.documentation.Tags; +import org.apache.nifi.annotation.documentation.WritesAttribute; import org.apache.nifi.annotation.lifecycle.OnScheduled; import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.components.PropertyValue; @@ -36,8 +44,6 @@ import org.apache.nifi.components.ValidationResult; import org.apache.nifi.components.Validator; import org.apache.nifi.flowfile.FlowFile; import org.apache.nifi.flowfile.attributes.CoreAttributes; -import org.apache.nifi.stream.io.BufferedInputStream; -import org.apache.nifi.stream.io.StreamUtils; import org.apache.nifi.processor.DataUnit; import org.apache.nifi.processor.ProcessContext; import org.apache.nifi.processor.ProcessSession; @@ -45,20 +51,18 @@ import org.apache.nifi.processor.Relationship; import org.apache.nifi.processor.exception.ProcessException; import org.apache.nifi.processor.io.InputStreamCallback; import org.apache.nifi.processor.util.StandardValidators; +import org.apache.nifi.stream.io.BufferedInputStream; +import org.apache.nifi.stream.io.StreamUtils; import org.apache.nifi.util.StopWatch; import org.apache.nifi.util.Tuple; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FSDataOutputStream; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.fs.permission.FsPermission; -import org.apache.hadoop.ipc.RemoteException; /** * This processor copies FlowFiles to HDFS. */ @Tags({"hadoop", "HDFS", "put", "copy", "filesystem"}) @CapabilityDescription("Write FlowFile data to Hadoop Distributed File System (HDFS)") +@WritesAttribute(attribute = "filename", description = "The name of the file written to HDFS comes from the value of this attribute.") +@SeeAlso(GetHDFS.class) public class PutHDFS extends AbstractHadoopProcessor { public static final String REPLACE_RESOLUTION = "replace"; diff --git a/nifi/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/java/org/apache/nifi/processors/kafka/GetKafka.java b/nifi/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/java/org/apache/nifi/processors/kafka/GetKafka.java index 5383e9ded5..4f894f93b7 100644 --- a/nifi/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/java/org/apache/nifi/processors/kafka/GetKafka.java +++ b/nifi/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/java/org/apache/nifi/processors/kafka/GetKafka.java @@ -42,6 +42,8 @@ import kafka.message.MessageAndMetadata; import org.apache.nifi.annotation.behavior.SupportsBatching; import org.apache.nifi.annotation.documentation.CapabilityDescription; import org.apache.nifi.annotation.documentation.Tags; +import org.apache.nifi.annotation.documentation.WritesAttribute; +import org.apache.nifi.annotation.documentation.WritesAttributes; import org.apache.nifi.annotation.lifecycle.OnScheduled; import org.apache.nifi.annotation.lifecycle.OnStopped; import org.apache.nifi.annotation.lifecycle.OnUnscheduled; @@ -59,6 +61,10 @@ import org.apache.nifi.processor.util.StandardValidators; @SupportsBatching @CapabilityDescription("Fetches messages from Apache Kafka") @Tags({"Kafka", "Apache", "Get", "Ingest", "Ingress", "Topic", "PubSub"}) +@WritesAttributes({ @WritesAttribute(attribute = "kafka.topic", description = "The name of the Kafka Topic from which the message was received"), + @WritesAttribute(attribute = "kafka.key", description = "The 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"), + @WritesAttribute(attribute = "kafka.partition", description = "The partition of the Kafka Topic from which the message was received. This attribute is added only if the batch size is 1"), + @WritesAttribute(attribute = "kafka.offset", description = "The offset of the message within the Kafka partition. This attribute is added only if the batch size is 1") }) public class GetKafka extends AbstractProcessor { public static final PropertyDescriptor ZOOKEEPER_CONNECTION_STRING = new PropertyDescriptor.Builder() .name("ZooKeeper Connection String") From ff1ba78b342ec7a7780723e3a7f58a81e11ab7e0 Mon Sep 17 00:00:00 2001 From: danbress Date: Thu, 5 Mar 2015 14:53:31 -0500 Subject: [PATCH 045/116] NIFI-309 fixing bug when dealing with string class names --- .../apache/nifi/documentation/html/HtmlDocumentationWriter.java | 2 +- 1 file changed, 1 insertion(+), 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 f3c0edf742..9856c5dc8d 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 @@ -167,7 +167,7 @@ public class HtmlDocumentationWriter implements DocumentationWriter { final String link = "../" + linkedComponent + "/index.html"; - final int indexOfLastPeriod = Math.max(0, linkedComponent.lastIndexOf(".")); + final int indexOfLastPeriod = linkedComponent.lastIndexOf(".") + 1; writeLink(xmlStreamWriter, linkedComponent.substring(indexOfLastPeriod), link); From 68dffdd1d881af617ef95b2980b80ed1ac9fdf75 Mon Sep 17 00:00:00 2001 From: danbress Date: Fri, 6 Mar 2015 09:56:44 -0500 Subject: [PATCH 046/116] NIFI-309 Updating additionalDetails.html or removing based what is contained in new annotations --- .../additionalDetails.html | 7 -- .../additionalDetails.html | 57 --------- .../additionalDetails.html | 32 ----- .../additionalDetails.html | 51 -------- .../additionalDetails.html | 33 ----- .../additionalDetails.html | 68 ----------- .../additionalDetails.html | 58 --------- .../additionalDetails.html | 34 ------ .../additionalDetails.html | 18 +-- .../additionalDetails.html | 57 --------- .../additionalDetails.html | 74 ----------- .../additionalDetails.html | 79 ------------ .../additionalDetails.html | 45 ------- .../additionalDetails.html | 80 ------------ .../additionalDetails.html | 35 ------ .../additionalDetails.html | 35 ------ .../additionalDetails.html | 51 +++++--- .../additionalDetails.html | 64 ---------- .../additionalDetails.html | 115 ------------------ .../additionalDetails.html | 49 -------- .../additionalDetails.html | 48 -------- .../additionalDetails.html | 34 ------ .../additionalDetails.html | 45 ------- .../additionalDetails.html | 82 ------------- .../additionalDetails.html | 67 ---------- .../additionalDetails.html | 72 ----------- .../additionalDetails.html | 106 ---------------- .../additionalDetails.html | 7 -- .../additionalDetails.html | 31 ----- .../additionalDetails.html | 7 -- 30 files changed, 38 insertions(+), 1503 deletions(-) delete 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.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.PutHDFS/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/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/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/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/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/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/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/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/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/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/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/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/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/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/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/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/additionalDetails.html delete mode 100644 nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.SegmentContent/additionalDetails.html delete mode 100644 nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.SplitContent/additionalDetails.html delete mode 100644 nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.SplitText/additionalDetails.html delete mode 100644 nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.UnpackContent/additionalDetails.html delete mode 100644 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 diff --git a/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/resources/docs/org.apache.nifi.processors.hadoop.CreateHadoopSequenceFile/additionalDetails.html b/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/resources/docs/org.apache.nifi.processors.hadoop.CreateHadoopSequenceFile/additionalDetails.html index d9403822b8..3c518fbe1c 100644 --- a/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/resources/docs/org.apache.nifi.processors.hadoop.CreateHadoopSequenceFile/additionalDetails.html +++ b/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/resources/docs/org.apache.nifi.processors.hadoop.CreateHadoopSequenceFile/additionalDetails.html @@ -42,12 +42,5 @@

    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. - -

    - 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 deleted file mode 100644 index cd8cc82d75..0000000000 --- a/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/resources/docs/org.apache.nifi.processors.hadoop.GetHDFS/additionalDetails.html +++ /dev/null @@ -1,57 +0,0 @@ - - - - - - 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.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 deleted file mode 100644 index ca0b7cdfbe..0000000000 --- a/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/resources/docs/org.apache.nifi.processors.hadoop.GetHDFSSequenceFile/additionalDetails.html +++ /dev/null @@ -1,32 +0,0 @@ - - - - - - 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.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 deleted file mode 100644 index 17b286b3dd..0000000000 --- a/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/resources/docs/org.apache.nifi.processors.hadoop.PutHDFS/additionalDetails.html +++ /dev/null @@ -1,51 +0,0 @@ - - - - - - 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-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 b182189c97..60611b6003 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 @@ -41,38 +41,5 @@ 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-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 deleted file mode 100644 index a85456df7e..0000000000 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.CompressContent/additionalDetails.html +++ /dev/null @@ -1,68 +0,0 @@ - - - - - - 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.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 deleted file mode 100644 index 188b137af9..0000000000 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.DetectDuplicate/additionalDetails.html +++ /dev/null @@ -1,58 +0,0 @@ - - - - - - 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.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 deleted file mode 100644 index 5fe58a5b26..0000000000 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.EvaluateXPath/additionalDetails.html +++ /dev/null @@ -1,34 +0,0 @@ - - - - - - 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.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 ae9e6ff3f7..99c530e76a 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 @@ -27,21 +27,9 @@ -

    - Modifies Attributes: -

    - -

    - 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 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 deleted file mode 100644 index 2b1450a01c..0000000000 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.ExecuteStreamCommand/additionalDetails.html +++ /dev/null @@ -1,57 +0,0 @@ - - - - - - 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.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 deleted file mode 100644 index 4805ed8936..0000000000 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.GetFTP/additionalDetails.html +++ /dev/null @@ -1,74 +0,0 @@ - - - - - - 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.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 deleted file mode 100644 index 6a3b77f012..0000000000 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.GetFile/additionalDetails.html +++ /dev/null @@ -1,79 +0,0 @@ - - - - - - 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.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 deleted file mode 100644 index c975303463..0000000000 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.GetHTTP/additionalDetails.html +++ /dev/null @@ -1,45 +0,0 @@ - - - - - - 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.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 deleted file mode 100644 index 74a7cb0d14..0000000000 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.GetSFTP/additionalDetails.html +++ /dev/null @@ -1,80 +0,0 @@ - - - - - - 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.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 deleted file mode 100644 index 165568c5a4..0000000000 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.HashAttribute/additionalDetails.html +++ /dev/null @@ -1,35 +0,0 @@ - - - - - - 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.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 deleted file mode 100644 index 8a3d1cbd7f..0000000000 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.HashContent/additionalDetails.html +++ /dev/null @@ -1,35 +0,0 @@ - - - - - - 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.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 f993fee963..b5a31fc073 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 @@ -25,20 +25,39 @@

    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.
    - + +

    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
    • +
    + 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 deleted file mode 100644 index 9a18355b11..0000000000 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.InvokeHTTP/additionalDetails.html +++ /dev/null @@ -1,64 +0,0 @@ - - - - - - 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.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 deleted file mode 100644 index 22007648c1..0000000000 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.MergeContent/additionalDetails.html +++ /dev/null @@ -1,115 +0,0 @@ - - - - - - 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.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 deleted file mode 100644 index d55fcec4d8..0000000000 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.MonitorActivity/additionalDetails.html +++ /dev/null @@ -1,49 +0,0 @@ - - - - - - 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.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 deleted file mode 100644 index d193e93be9..0000000000 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.PostHTTP/additionalDetails.html +++ /dev/null @@ -1,48 +0,0 @@ - - - - - - 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.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 deleted file mode 100644 index 93876b62e7..0000000000 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.PutSFTP/additionalDetails.html +++ /dev/null @@ -1,34 +0,0 @@ - - - - - - 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.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 deleted file mode 100644 index aff65491d3..0000000000 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.ScanContent/additionalDetails.html +++ /dev/null @@ -1,45 +0,0 @@ - - - - - - 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.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 deleted file mode 100644 index 957913f0e5..0000000000 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.SegmentContent/additionalDetails.html +++ /dev/null @@ -1,82 +0,0 @@ - - - - - - SegmentContent - - - - - - - -

    - Adds or Modifies Attributes: -

    - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
    Attribute NameDescription
    segment.identifierAll segments produced from the same parent FlowFile will have the same randomly generated UUID added for this attribute. This attribute is added to maintain backward compatibility, but the fragment.identifier is preferred, as it is designed to work in conjunction with the MergeContent Processor.
    segment.indexA one-up number that indicates the ordering of the segments that were created from a single parent FlowFile. This attribute is added to maintain backward compatibility, but the fragment.index is preferred, as it is designed to work in conjunction with the MergeContent Processor.
    segment.countThe number of segments generated from the parent FlowFile. This attribute is added to maintain backward compatibility, but the fragment.count is preferred, as it is designed to work in conjunction with the MergeContent Processor.
    fragment.identifierAll segments produced from the same parent FlowFile will have the same randomly generated UUID added for this attribute.
    fragment.indexA one-up number that indicates the ordering of the segments that were created from a single parent FlowFile.
    fragment.countThe number of segments generated from the parent FlowFile.
    segment.original.filenameThe filename of the parent FlowFile
    filenameThe filename will be updated to include the parent's filename, the segment index, and the segment count.
    -

    - See Also: -

    -
      -
    • MergeContent - can defragment FlowFiles - that have been segmented.
    • -
    - - 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 deleted file mode 100644 index 683b0fa5a1..0000000000 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.SplitContent/additionalDetails.html +++ /dev/null @@ -1,67 +0,0 @@ - - - - - - SplitContent - - - - - - - -

    - Modifies Attributes: -

    - - - - - - - - - - - - - - - - - - - - - - - - - -
    Attribute NameDescription
    fragment.identifierAll split FlowFiles produced from the same parent FlowFile will have the same randomly generated - UUID added for this attribute.
    fragment.indexA one-up number that indicates the ordering of the split FlowFiles that were created from a - single parent FlowFile.
    fragment.countThe number of split FlowFiles generated from the parent FlowFile.
    segment.original.filenameThe filename of the parent FlowFile. -
    -

    - See Also: -

    -
      -
    • MergeContent - can defragment FlowFiles - that have been split if Keep Byte Sequence is set to true.
    • -
    - - 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 deleted file mode 100644 index ad01242dc0..0000000000 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.SplitText/additionalDetails.html +++ /dev/null @@ -1,72 +0,0 @@ - - - - - - SplitText - - - - - - - -

    - Modifies Attributes: -

    - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
    Attribute NameDescription
    text.line.countThe number of lines of text from the original FlowFile that were copied to this FlowFile.
    fragment.identifierAll split FlowFiles produced from the same parent FlowFile will have the same randomly generated - UUID added for this attribute.
    fragment.indexA one-up number that indicates the ordering of the split FlowFiles that were created from a - single parent FlowFile.
    fragment.countThe number of split FlowFiles generated from the parent FlowFile.
    segment.original.filenameThe filename of the parent FlowFile. -
    - -

    - See Also: -

    -
      -
    • MergeContent - can defragment FlowFiles - that have been split if Remove Trailing Newlines is set to false and Header Line Count is set to 0.
    • -
    - - 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 deleted file mode 100644 index 30ac301fad..0000000000 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.UnpackContent/additionalDetails.html +++ /dev/null @@ -1,106 +0,0 @@ - - - - - - UnpackContent - - - - - - - -

    - Uses Attributes: -

    - - - - - - - - - - - - - -
    Attribute NameDescription
    mime.type - If the <Packaging Format> property is set to use mime.type attribute, - this attribute is used to determine the FlowFile's MIME Type. - In this case, if the attribute is set to application/tar, the TAR Packaging Format will be used. - If the attribute is set to application/zip, the ZIP Packaging Format will be used. - If the attribute is set to application/flowfile-v3 or application/flowfile-v2 or application/flowfile-v1, - the appropriate FlowFile Packaging Format will be used. - If this attribute is missing, the FlowFile will be routed to 'failure'. - Otherwise, if the attribute's value is not one of those mentioned above, the FlowFile will be - routed to 'success' without being unpacked. -
    - -

    - Modifies Attributes: -

    - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
    Attribute NameDescription
    mime.type - If the FlowFile is successfully unpacked, its MIME Type is no longer known, so the - mime.type attribute is set to application/octet-stream. -
    fragment.identifierAll unpacked FlowFiles produced from the same parent FlowFile will have the same randomly generated - UUID added for this attribute.
    fragment.indexA one-up number that indicates the ordering of the unpacked FlowFiles that were created from a - single parent FlowFile.
    fragment.countThe number of unpacked FlowFiles generated from the parent FlowFile.
    segment.original.filenameThe filename of the parent FlowFile. Extensions of .tar, .zip or .pkg are removed because the MergeContent processor automatically adds - those extensions if it is used to rebuild the original FlowFile. -
    - -

    - See Also: -

    -
      -
    • MergeContent - can repackage FlowFiles - that have been unpacked.
    • -
    - - 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/additionalDetails.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 index a3bc60f043..4cde8c620a 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/additionalDetails.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 @@ -41,12 +41,5 @@ </service> </services> - - See Also: - - 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/additionalDetails.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 deleted file mode 100644 index fa1bf42100..0000000000 --- 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/additionalDetails.html +++ /dev/null @@ -1,31 +0,0 @@ - - - - - - Distributed Map Cache Client Service - - - - - See Also: - - - - 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/additionalDetails.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 index 2171be246e..ad9822173f 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/additionalDetails.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 @@ -42,12 +42,5 @@ </service> </services> - - See Also: - - From 201cfeaf1c9a79d0607450b1f085c33876a7cd57 Mon Sep 17 00:00:00 2001 From: danbress Date: Fri, 6 Mar 2015 10:20:25 -0500 Subject: [PATCH 047/116] NIFI-309 cleaning up formatting of MergeContent --- .../org/apache/nifi/processors/standard/MergeContent.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/MergeContent.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/MergeContent.java index 187c50a19a..e19fc60614 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/MergeContent.java +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/MergeContent.java @@ -80,10 +80,10 @@ import org.apache.nifi.util.ObjectHolder; @ReadsAttribute(attribute = "fragment.count", description = "Applicable only if the 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"), @ReadsAttribute(attribute = "segment.original.filename", description = "Applicable only if the 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"), @ReadsAttribute(attribute = "tar.permissions", description = "Applicable only if the 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") }) -@WritesAttributes({ @WritesAttribute(attribute = "filename", description = "When 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
    "), +@WritesAttributes({ @WritesAttribute(attribute = "filename", description = "When 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"), @WritesAttribute(attribute = "merge.count", description = "The number of FlowFiles that were merged into this bundle"), @WritesAttribute(attribute = "merge.bin.age", description = "The 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") }) @SeeAlso(SegmentContent.class) From 90fd0edaf85e3d175a5908760d1c841b4ac97096 Mon Sep 17 00:00:00 2001 From: danbress Date: Sun, 8 Mar 2015 12:00:11 -0400 Subject: [PATCH 048/116] NIFI-309 documenting @SeeAlso, @ReadsAttributes, @WritesAttributes --- .../src/main/asciidoc/developer-guide.adoc | 35 +++++++++++++++++++ 1 file changed, 35 insertions(+) diff --git a/nifi/nifi-docs/src/main/asciidoc/developer-guide.adoc b/nifi/nifi-docs/src/main/asciidoc/developer-guide.adoc index d8b5d5048b..35a4eb8cb5 100644 --- a/nifi/nifi-docs/src/main/asciidoc/developer-guide.adoc +++ b/nifi/nifi-docs/src/main/asciidoc/developer-guide.adoc @@ -656,6 +656,41 @@ public static final ExampleProcessor extends Processor { } ---- +=== Documenting FlowFile Attribute Interaction + +Many times a processor will expect certain FlowFile attributes be set on in-bound FlowFiles in order +for the processor to function properly. In other cases a processor may update or +create FlowFile attributes on the out-bound FlowFile. Processor developers may document both of these +behaviors using the `ReadsAttribute` and `WritesAttribute` documentation annotations. These attributes are used to generate documentation +that gives users a better understanding of how a processor will interact with the flow. + +Note: Because Java 7 does not support +repeated annotations on a type, you may need to use `ReadsAttributes` and `WritesAttributes` to indicate +that a processor reads or writes multiple FlowFile attributes. This annotation can only be applied to Processors. An example is listed below: + +[source, java] +---- +@WritesAttributes({ @WritesAttribute(attribute = "invokehttp.status.code", description = "The status code that is returned"), + @WritesAttribute(attribute = "invokehttp.status.message", description = "The status message that is returned"), + @WritesAttribute(attribute = "invokehttp.response.body", description = "The response body"), + @WritesAttribute(attribute = "invokehttp.request.url", description = "The request URL"), + @WritesAttribute(attribute = "invokehttp.tx.id", description = "The transaction ID that is returned after reading the response"), + @WritesAttribute(attribute = "invokehttp.remote.dn", description = "The DN of the remote server") }) +public final class InvokeHTTP extends AbstractProcessor { +---- + +=== Documenting Related Components +Often Processors and ControllerServices are related to one another. Sometimes its a put/get relation as in `PutFile` and `GetFile`. +Sometimes a Processor uses a ControllerService like `InvokeHTTP` and `StandardSSLContextService`. Sometimes one ControllerService uses another +like `DistributedMapCacheClientService` and `DistributedMapCacheServer`. Developers of these extension points may relate these +different components using the `SeeAlso` tag. This annotation links these components in the documentation. +`SeeAlso` can be applied to Processors, ControllerServices and ReportingTasks. An example of how to do this is listed below: + +[source, java] +---- +@SeeAlso(GetFile.class) +public class PutFile extends AbstractProcessor { +---- === Advanced Documentation From 72153c94532fc021fd973d3d8c31ed725beddd68 Mon Sep 17 00:00:00 2001 From: danbress Date: Sun, 8 Mar 2015 12:03:27 -0400 Subject: [PATCH 049/116] NIFI-309 adding javadoc --- .../nifi/annotation/documentation/ReadsAttribute.java | 9 +++++++++ .../nifi/annotation/documentation/ReadsAttributes.java | 4 ++++ .../nifi/annotation/documentation/WritesAttribute.java | 10 ++++++++++ .../annotation/documentation/WritesAttributes.java | 4 ++++ 4 files changed, 27 insertions(+) diff --git a/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/documentation/ReadsAttribute.java b/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/documentation/ReadsAttribute.java index d644d94df4..546db2d60e 100644 --- a/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/documentation/ReadsAttribute.java +++ b/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/documentation/ReadsAttribute.java @@ -36,6 +36,15 @@ import java.lang.annotation.Target; @Retention(RetentionPolicy.RUNTIME) @Inherited public @interface ReadsAttribute { + /** + * The FlowFile attribute that is being read + * @return + */ public String attribute(); + + /** + * The description of how the attribute is being used + * @return + */ public String description() default ""; } diff --git a/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/documentation/ReadsAttributes.java b/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/documentation/ReadsAttributes.java index 9edd06fc8f..78ba4ecb94 100644 --- a/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/documentation/ReadsAttributes.java +++ b/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/documentation/ReadsAttributes.java @@ -35,5 +35,9 @@ import java.lang.annotation.Target; @Retention(RetentionPolicy.RUNTIME) @Inherited public @interface ReadsAttributes { + /** + * A list of attributes that may be read + * @return + */ public ReadsAttribute[] value(); } diff --git a/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/documentation/WritesAttribute.java b/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/documentation/WritesAttribute.java index 16ef56b8db..43f4947677 100644 --- a/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/documentation/WritesAttribute.java +++ b/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/documentation/WritesAttribute.java @@ -36,6 +36,16 @@ import java.lang.annotation.Target; @Retention(RetentionPolicy.RUNTIME) @Inherited public @interface WritesAttribute { + + /** + * The FlowFile attribute that is being created or updated + * @return + */ public String attribute(); + + /** + * A description of what is being written to the FlowFile attribute + * @return + */ public String description() default ""; } diff --git a/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/documentation/WritesAttributes.java b/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/documentation/WritesAttributes.java index d9d238028a..140c85d0cb 100644 --- a/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/documentation/WritesAttributes.java +++ b/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/documentation/WritesAttributes.java @@ -36,5 +36,9 @@ import java.lang.annotation.Target; @Retention(RetentionPolicy.RUNTIME) @Inherited public @interface WritesAttributes { + /** + * A list of FlowFile attributes that may be written or updated + * @return + */ public WritesAttribute[] value(); } From bae2b4069b38e32b4b9243e314597f48633c704d Mon Sep 17 00:00:00 2001 From: danbress Date: Wed, 18 Mar 2015 13:41:57 -0400 Subject: [PATCH 050/116] NIFI-309 - Adding DynamicProperty support --- .../behavior/DynamicProperties.java | 26 +++++++ .../annotation/behavior/DynamicProperty.java | 45 ++++++++++++ .../html/HtmlDocumentationWriter.java | 70 +++++++++++++++++++ .../example/FullyDocumentedProcessor.java | 2 + 4 files changed, 143 insertions(+) create mode 100644 nifi/nifi-api/src/main/java/org/apache/nifi/annotation/behavior/DynamicProperties.java create mode 100644 nifi/nifi-api/src/main/java/org/apache/nifi/annotation/behavior/DynamicProperty.java diff --git a/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/behavior/DynamicProperties.java b/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/behavior/DynamicProperties.java new file mode 100644 index 0000000000..4fdfd080b3 --- /dev/null +++ b/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/behavior/DynamicProperties.java @@ -0,0 +1,26 @@ +package org.apache.nifi.annotation.behavior; + +import java.lang.annotation.Documented; +import java.lang.annotation.ElementType; +import java.lang.annotation.Inherited; +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; +import java.lang.annotation.Target; + +/** + * Indicates that a component has more than one dynamic property + * + * @author + * + */ +@Documented +@Target({ ElementType.TYPE }) +@Retention(RetentionPolicy.RUNTIME) +@Inherited +public @interface DynamicProperties { + /** + * A list of the dynamic properties supported by a component + * @return A list of the dynamic properties supported by a component + */ + public DynamicProperty[] value(); +} diff --git a/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/behavior/DynamicProperty.java b/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/behavior/DynamicProperty.java new file mode 100644 index 0000000000..7b2150dfef --- /dev/null +++ b/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/behavior/DynamicProperty.java @@ -0,0 +1,45 @@ +package org.apache.nifi.annotation.behavior; + +import java.lang.annotation.Documented; +import java.lang.annotation.ElementType; +import java.lang.annotation.Inherited; +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; +import java.lang.annotation.Target; + +import org.apache.nifi.components.ConfigurableComponent; + +/** + * An annotation that may be placed on a {@link ConfigurableComponent} to + * indicate that it supports a dynamic property. + * + * @author + * + */ +@Documented +@Target({ ElementType.TYPE }) +@Retention(RetentionPolicy.RUNTIME) +@Inherited +public @interface DynamicProperty { + /** + * A description of what the name of the dynamic property may be + * + * @return A description of what the name of the dynamic property may be + */ + public String name(); + + /** + * Indicates whether or not the dynamic property supports expression + * language + * + * @return whether or not the dynamic property supports expression + * language + */ + public boolean supportsExpressionLanguage() default false; + + /** + * Provides a description of what the meaning of the property is, and what the expected values are + * @return a description of what the meaning of the property is, and what the expected values are + */ + public String description(); +} 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 9856c5dc8d..7def25c712 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 @@ -18,6 +18,7 @@ package org.apache.nifi.documentation.html; import java.io.IOException; import java.io.OutputStream; +import java.util.ArrayList; import java.util.List; import javax.xml.stream.FactoryConfigurationError; @@ -26,6 +27,8 @@ import javax.xml.stream.XMLStreamException; import javax.xml.stream.XMLStreamWriter; import org.apache.commons.lang3.StringUtils; +import org.apache.nifi.annotation.behavior.DynamicProperties; +import org.apache.nifi.annotation.behavior.DynamicProperty; import org.apache.nifi.annotation.documentation.CapabilityDescription; import org.apache.nifi.annotation.documentation.SeeAlso; import org.apache.nifi.annotation.documentation.Tags; @@ -126,6 +129,7 @@ public class HtmlDocumentationWriter implements DocumentationWriter { writeDescription(configurableComponent, xmlStreamWriter, hasAdditionalDetails); writeTags(configurableComponent, xmlStreamWriter); writeProperties(configurableComponent, xmlStreamWriter); + writeDynamicProperties(configurableComponent, xmlStreamWriter); writeAdditionalBodyInfo(configurableComponent, xmlStreamWriter); writeSeeAlso(configurableComponent, xmlStreamWriter); xmlStreamWriter.writeEndElement(); @@ -350,7 +354,73 @@ public class HtmlDocumentationWriter implements DocumentationWriter { writeSimpleElement(xmlStreamWriter, "p", "This component has no required or optional properties."); } } + + /** + * Writes a list of the dynamic properties that a processor supports + * @param configurableComponent + * @param xmlStreamWriter + * @throws XMLStreamException + */ + private void writeDynamicProperties(final ConfigurableComponent configurableComponent, + final XMLStreamWriter xmlStreamWriter) throws XMLStreamException { + final List dynamicProperties = getDynamicProperties(configurableComponent); + + if (dynamicProperties != null && dynamicProperties.size() > 0) { + writeSimpleElement(xmlStreamWriter, "h3", "Dynamic Properties: "); + xmlStreamWriter.writeStartElement("p"); + xmlStreamWriter + .writeCharacters("Dynamic Properties allow the user to specify both the name and value of a property."); + xmlStreamWriter.writeStartElement("table"); + xmlStreamWriter.writeStartElement("tr"); + writeSimpleElement(xmlStreamWriter, "th", "Name"); + writeSimpleElement(xmlStreamWriter, "th", "Description"); + xmlStreamWriter.writeEndElement(); + for (final DynamicProperty dynamicProperty : dynamicProperties) { + xmlStreamWriter.writeStartElement("tr"); + writeSimpleElement(xmlStreamWriter, "td", dynamicProperty.name()); + xmlStreamWriter.writeStartElement("td"); + xmlStreamWriter.writeCharacters(dynamicProperty.description()); + if (dynamicProperty.supportsExpressionLanguage()) { + xmlStreamWriter.writeEmptyElement("br"); + writeSimpleElement(xmlStreamWriter, "strong", "Supports Expression Language: true"); + } + xmlStreamWriter.writeEndElement(); + xmlStreamWriter.writeEndElement(); + } + + xmlStreamWriter.writeEndElement(); + xmlStreamWriter.writeEndElement(); + } + } + + /** + * Gets the dynamic properties for a configurable component + * @param configurableComponent + * @return + */ + private List getDynamicProperties(ConfigurableComponent configurableComponent) { + final List dynamicProperties = new ArrayList<>(); + final DynamicProperties dynProps = configurableComponent.getClass().getAnnotation(DynamicProperties.class); + if (dynProps != null) { + for (final DynamicProperty dynProp : dynProps.value()) { + dynamicProperties.add(dynProp); + } + } + + final DynamicProperty dynProp = configurableComponent.getClass().getAnnotation(DynamicProperty.class); + dynamicProperties.add(dynProp); + + return dynamicProperties; + } + + /** + * Writes an info icon with a description. + * + * @param xmlStreamWriter + * @param description the description of the item + * @throws XMLStreamException + */ private void writeValidValueDescription(XMLStreamWriter xmlStreamWriter, String description) throws XMLStreamException { xmlStreamWriter.writeCharacters(" "); 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 84dc88f5e6..1814f51e6e 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.annotation.behavior.DynamicProperty; import org.apache.nifi.annotation.documentation.CapabilityDescription; import org.apache.nifi.annotation.documentation.ReadsAttribute; import org.apache.nifi.annotation.documentation.SeeAlso; @@ -43,6 +44,7 @@ import org.apache.nifi.processor.util.StandardValidators; @WritesAttributes({@WritesAttribute(attribute="first", description="this is the first attribute i write"), @WritesAttribute(attribute="second")}) @ReadsAttribute(attribute = "incoming", description="this specifies the format of the thing") @SeeAlso(value={FullyDocumentedControllerService.class, FullyDocumentedReportingTask.class}, classNames={"org.apache.nifi.processor.ExampleProcessor"}) +@DynamicProperty(name="the relationship to route to", supportsExpressionLanguage=true, description="some XPath") public class FullyDocumentedProcessor extends AbstractProcessor { public static final PropertyDescriptor DIRECTORY = new PropertyDescriptor.Builder().name("Input Directory") From cce69a2f947cd66d16fcb154692f9d411dd4c815 Mon Sep 17 00:00:00 2001 From: danbress Date: Wed, 18 Mar 2015 13:59:13 -0400 Subject: [PATCH 051/116] NIFI-309 supporting DynamicRelationships --- .../documentation/DynamicRelationships.java | 40 ++++++++++++++++ .../HtmlProcessorDocumentationWriter.java | 46 ++++++++++++++++++- .../example/FullyDocumentedProcessor.java | 2 + 3 files changed, 86 insertions(+), 2 deletions(-) create mode 100644 nifi/nifi-api/src/main/java/org/apache/nifi/annotation/documentation/DynamicRelationships.java diff --git a/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/documentation/DynamicRelationships.java b/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/documentation/DynamicRelationships.java new file mode 100644 index 0000000000..0047cb13b8 --- /dev/null +++ b/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/documentation/DynamicRelationships.java @@ -0,0 +1,40 @@ +package org.apache.nifi.annotation.documentation; + +import java.lang.annotation.Documented; +import java.lang.annotation.ElementType; +import java.lang.annotation.Inherited; +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; +import java.lang.annotation.Target; + +import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.processor.Processor; +import org.apache.nifi.processor.Relationship; + +/** + * Annotation to indicate that a {@link Processor} supports dynamic + * relationship. A dynamic {@link Relationship} is one where the relationship is + * generated based on a user defined {@link PropertyDescriptor} + * + * @author + * + */ +@Documented +@Target({ ElementType.TYPE }) +@Retention(RetentionPolicy.RUNTIME) +@Inherited +public @interface DynamicRelationships { + /** + * Describes the name(s) of the dynamic relationship(s) + * + * @return a description of the name(s) of the dynamic relationship(s) + */ + public String name(); + + /** + * Describes the data that should be routed to the dynamic relationship(s) + * + * @return a description the data that should be routed to the dynamic relationship(s) + */ + public String description(); +} 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 af218d879d..6eed4dcbf5 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 @@ -23,6 +23,7 @@ import javax.xml.stream.XMLStreamException; import javax.xml.stream.XMLStreamWriter; import org.apache.commons.lang3.StringUtils; +import org.apache.nifi.annotation.documentation.DynamicRelationships; import org.apache.nifi.annotation.documentation.ReadsAttribute; import org.apache.nifi.annotation.documentation.ReadsAttributes; import org.apache.nifi.annotation.documentation.WritesAttribute; @@ -42,8 +43,9 @@ public class HtmlProcessorDocumentationWriter extends HtmlDocumentationWriter { @Override protected void writeAdditionalBodyInfo(final ConfigurableComponent configurableComponent, final XMLStreamWriter xmlStreamWriter) throws XMLStreamException { - final Processor processor = (Processor) configurableComponent; + final Processor processor = (Processor) configurableComponent; writeRelationships(processor, xmlStreamWriter); + writeDynamicRelationships(processor, xmlStreamWriter); writeAttributeInfo(processor, xmlStreamWriter); } @@ -199,7 +201,7 @@ public class HtmlProcessorDocumentationWriter extends HtmlDocumentationWriter { */ private void writeRelationships(final Processor processor, final XMLStreamWriter xmlStreamWriter) throws XMLStreamException { - + writeSimpleElement(xmlStreamWriter, "h3", "Relationships: "); if (processor.getRelationships().size() > 0) { @@ -220,4 +222,44 @@ public class HtmlProcessorDocumentationWriter extends HtmlDocumentationWriter { xmlStreamWriter.writeCharacters("This processor has no relationships."); } } + + /** + * Writes dynamic relationship information + * + * @param processor + * @param xmlStreamWriter + * @throws XMLStreamException + */ + private void writeDynamicRelationships(final Processor processor, final XMLStreamWriter xmlStreamWriter) throws XMLStreamException { + + List dynamicRelationships = getDynamicRelationships(processor); + + if (dynamicRelationships.size() > 0) { + writeSimpleElement(xmlStreamWriter, "h3", "Dynamic Relationships: "); + xmlStreamWriter.writeStartElement("table"); + xmlStreamWriter.writeStartElement("tr"); + writeSimpleElement(xmlStreamWriter, "th", "Name"); + writeSimpleElement(xmlStreamWriter, "th", "Description"); + xmlStreamWriter.writeEndElement(); + + for (DynamicRelationships dynamicRelationship : dynamicRelationships) { + xmlStreamWriter.writeStartElement("tr"); + writeSimpleElement(xmlStreamWriter, "td", dynamicRelationship.name()); + writeSimpleElement(xmlStreamWriter, "td", dynamicRelationship.description()); + xmlStreamWriter.writeEndElement(); + } + xmlStreamWriter.writeEndElement(); + } + } + + private List getDynamicRelationships(Processor processor) { + List results = new ArrayList<>(); + + DynamicRelationships dynamicRelationships = processor.getClass().getAnnotation(DynamicRelationships.class); + if (dynamicRelationships != null) { + results.add(dynamicRelationships); + } + + return results; + } } 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 1814f51e6e..782d30548e 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 @@ -24,6 +24,7 @@ import java.util.Set; import org.apache.nifi.annotation.behavior.DynamicProperty; import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.documentation.DynamicRelationships; import org.apache.nifi.annotation.documentation.ReadsAttribute; import org.apache.nifi.annotation.documentation.SeeAlso; import org.apache.nifi.annotation.documentation.Tags; @@ -45,6 +46,7 @@ import org.apache.nifi.processor.util.StandardValidators; @ReadsAttribute(attribute = "incoming", description="this specifies the format of the thing") @SeeAlso(value={FullyDocumentedControllerService.class, FullyDocumentedReportingTask.class}, classNames={"org.apache.nifi.processor.ExampleProcessor"}) @DynamicProperty(name="the relationship to route to", supportsExpressionLanguage=true, description="some XPath") +@DynamicRelationships(name="name from dynamic property", description="all files that match the properties XPath") public class FullyDocumentedProcessor extends AbstractProcessor { public static final PropertyDescriptor DIRECTORY = new PropertyDescriptor.Builder().name("Input Directory") From a3e1d3a20bad9d8bcbb26244c0a6f03a38f01c57 Mon Sep 17 00:00:00 2001 From: danbress Date: Wed, 18 Mar 2015 15:23:34 -0400 Subject: [PATCH 052/116] NIFI-309 - applying @DynamicProperty and @DynamicRelationships to processors --- .../processors/standard/DistributeLoad.java | 21 +++++++++++-------- .../processors/standard/EvaluateJsonPath.java | 4 ++++ .../standard/EvaluateRegularExpression.java | 3 ++- .../processors/standard/EvaluateXPath.java | 2 ++ .../processors/standard/EvaluateXQuery.java | 2 ++ .../processors/standard/ExecuteProcess.java | 2 ++ .../standard/HandleHttpResponse.java | 2 ++ .../processors/standard/HashAttribute.java | 7 +++++++ .../nifi/processors/standard/InvokeHTTP.java | 3 +++ .../nifi/processors/standard/PutFTP.java | 4 ++++ .../nifi/processors/standard/PutSFTP.java | 2 ++ .../processors/standard/RouteOnAttribute.java | 14 ++++++++----- .../processors/standard/RouteOnContent.java | 4 ++++ .../processors/standard/TransformXml.java | 14 +++++++------ .../attributes/UpdateAttribute.java | 2 ++ 15 files changed, 65 insertions(+), 21 deletions(-) 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 c78251c5b4..a0dcc79072 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 @@ -29,6 +29,16 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; +import org.apache.commons.lang3.StringUtils; +import org.apache.nifi.annotation.behavior.DynamicProperty; +import org.apache.nifi.annotation.behavior.EventDriven; +import org.apache.nifi.annotation.behavior.SideEffectFree; +import org.apache.nifi.annotation.behavior.SupportsBatching; +import org.apache.nifi.annotation.behavior.TriggerWhenAnyDestinationAvailable; +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.documentation.DynamicRelationships; +import org.apache.nifi.annotation.documentation.Tags; +import org.apache.nifi.annotation.lifecycle.OnScheduled; import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.components.PropertyValue; import org.apache.nifi.components.ValidationContext; @@ -42,17 +52,8 @@ 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.behavior.EventDriven; -import org.apache.nifi.annotation.lifecycle.OnScheduled; -import org.apache.nifi.annotation.behavior.SideEffectFree; -import org.apache.nifi.annotation.behavior.SupportsBatching; -import org.apache.nifi.annotation.documentation.Tags; -import org.apache.nifi.annotation.behavior.TriggerWhenAnyDestinationAvailable; import org.apache.nifi.processor.util.StandardValidators; -import org.apache.commons.lang3.StringUtils; - @EventDriven @SideEffectFree @SupportsBatching @@ -62,6 +63,8 @@ import org.apache.commons.lang3.StringUtils; + "strategy, the default is to assign each destination a weighting of 1 (evenly distributed). However, optional properties" + "can be added to the change this; adding a property with the name '5' and value '10' means that the relationship with name " + "'5' will be receive 10 FlowFiles in each iteration instead of 1.") +@DynamicProperty(name="The relationship name(positive number)", description="The number of FlowFiles to this relationship per iteration") +@DynamicRelationships(name="A number 1..", description="FlowFiles are sent to this relationship per the ") public class DistributeLoad extends AbstractProcessor { public static final String STRATEGY_ROUND_ROBIN = "round robin"; diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateJsonPath.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateJsonPath.java index 64f6e0d428..e6356f0bfd 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateJsonPath.java +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateJsonPath.java @@ -20,11 +20,14 @@ import com.jayway.jsonpath.DocumentContext; import com.jayway.jsonpath.InvalidJsonException; import com.jayway.jsonpath.JsonPath; import com.jayway.jsonpath.PathNotFoundException; + import org.apache.commons.lang3.StringUtils; +import org.apache.nifi.annotation.behavior.DynamicProperty; import org.apache.nifi.annotation.behavior.EventDriven; import org.apache.nifi.annotation.behavior.SideEffectFree; import org.apache.nifi.annotation.behavior.SupportsBatching; import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.documentation.DynamicRelationships; import org.apache.nifi.annotation.documentation.Tags; import org.apache.nifi.annotation.lifecycle.OnRemoved; import org.apache.nifi.components.PropertyDescriptor; @@ -63,6 +66,7 @@ import java.util.concurrent.ConcurrentMap; + "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.'") +@DynamicProperty(name="A FlowFile attribute(if is set to 'flowfile-attribute')", description="JsonPath expression") public class EvaluateJsonPath extends AbstractJsonPathProcessor { public static final String DESTINATION_ATTRIBUTE = "flowfile-attribute"; diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateRegularExpression.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateRegularExpression.java index bb2e31a84c..e5598d4c4e 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateRegularExpression.java +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateRegularExpression.java @@ -40,13 +40,13 @@ 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.behavior.DynamicProperty; import org.apache.nifi.annotation.behavior.EventDriven; import org.apache.nifi.annotation.behavior.SideEffectFree; import org.apache.nifi.annotation.behavior.SupportsBatching; import org.apache.nifi.annotation.documentation.Tags; import org.apache.nifi.processor.io.InputStreamCallback; import org.apache.nifi.processor.util.StandardValidators; - import org.apache.commons.lang3.StringUtils; @EventDriven @@ -64,6 +64,7 @@ import org.apache.commons.lang3.StringUtils; + "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.") @Deprecated +@DynamicProperty(name="A FlowFile attribute", description="A regular expression with exactly one capturing group") public class EvaluateRegularExpression extends AbstractProcessor { public static final PropertyDescriptor CHARACTER_SET = new PropertyDescriptor.Builder() diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateXPath.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateXPath.java index d7ca00a581..5db3b0b864 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateXPath.java +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateXPath.java @@ -69,6 +69,7 @@ import org.apache.nifi.annotation.documentation.CapabilityDescription; import org.apache.nifi.annotation.documentation.WritesAttribute; import org.apache.nifi.annotation.behavior.EventDriven; import org.apache.nifi.annotation.lifecycle.OnScheduled; +import org.apache.nifi.annotation.behavior.DynamicProperty; import org.apache.nifi.annotation.behavior.SideEffectFree; import org.apache.nifi.annotation.behavior.SupportsBatching; import org.apache.nifi.annotation.documentation.Tags; @@ -93,6 +94,7 @@ import org.xml.sax.InputSource; + "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'") @WritesAttribute(attribute="user-defined", description="This processor adds user-defined attributes if the property is set to flowfile-attribute.") +@DynamicProperty(name="A FlowFile attribute(if is set to 'flowfile-attribute'", description="An XPath expression") public class EvaluateXPath extends AbstractProcessor { public static final String DESTINATION_ATTRIBUTE = "flowfile-attribute"; diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateXQuery.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateXQuery.java index 314ff9989e..143c6b4d5f 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateXQuery.java +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateXQuery.java @@ -50,6 +50,7 @@ import net.sf.saxon.s9api.XdmItem; import net.sf.saxon.s9api.XdmNode; import net.sf.saxon.s9api.XdmValue; +import org.apache.nifi.annotation.behavior.DynamicProperty; import org.apache.nifi.annotation.behavior.EventDriven; import org.apache.nifi.annotation.behavior.SideEffectFree; import org.apache.nifi.annotation.behavior.SupportsBatching; @@ -94,6 +95,7 @@ import org.xml.sax.InputSource; + "Destination is 'flowfile-attribute' and the XQueries matche nothing, no attributes will be applied to the " + "FlowFile.") @WritesAttribute(attribute="user-defined", description="This processor adds user-defined attributes if the property is set to flowfile-attribute .") +@DynamicProperty(name="A FlowFile attribute(if is set to 'flowfile-attribute'", description="An XQuery") public class EvaluateXQuery extends AbstractProcessor { public static final String DESTINATION_ATTRIBUTE = "flowfile-attribute"; diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ExecuteProcess.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ExecuteProcess.java index 31efd187d9..319cb2cd96 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ExecuteProcess.java +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ExecuteProcess.java @@ -42,6 +42,7 @@ import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.ReentrantLock; import org.apache.commons.lang3.StringUtils; +import org.apache.nifi.annotation.behavior.DynamicProperty; import org.apache.nifi.annotation.documentation.CapabilityDescription; import org.apache.nifi.annotation.documentation.Tags; import org.apache.nifi.annotation.lifecycle.OnScheduled; @@ -61,6 +62,7 @@ import org.apache.nifi.processor.util.StandardValidators; @CapabilityDescription("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.") +@DynamicProperty(name="An environment variable name", description="An environment variable value") public class ExecuteProcess extends AbstractProcessor { public static final PropertyDescriptor COMMAND = new PropertyDescriptor.Builder() diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/HandleHttpResponse.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/HandleHttpResponse.java index fd55927e42..a83b4ce169 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/HandleHttpResponse.java +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/HandleHttpResponse.java @@ -26,6 +26,7 @@ import java.util.regex.Pattern; import javax.servlet.http.HttpServletResponse; +import org.apache.nifi.annotation.behavior.DynamicProperty; import org.apache.nifi.annotation.documentation.CapabilityDescription; import org.apache.nifi.annotation.documentation.Tags; import org.apache.nifi.components.PropertyDescriptor; @@ -40,6 +41,7 @@ import org.apache.nifi.processor.util.StandardValidators; @Tags({"http", "https", "response", "egress", "web service"}) @CapabilityDescription("Sends an HTTP Response to the Requestor that generated a FlowFile. This Processor is designed to be used in conjunction with the HandleHttpRequest in order to create a web service.") +@DynamicProperty(name="An HTTP header name", description="An HTTP header value") public class HandleHttpResponse extends AbstractProcessor { public static final Pattern NUMBER_PATTERN = Pattern.compile("[0-9]+"); public static final String HTTP_CONTEXT_ID = "http.context.identifier"; diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/HashAttribute.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/HashAttribute.java index 932729168a..9c5c0b54d2 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/HashAttribute.java +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/HashAttribute.java @@ -31,6 +31,7 @@ import java.util.regex.Pattern; import org.apache.commons.codec.digest.DigestUtils; import org.apache.commons.lang3.StringUtils; +import org.apache.nifi.annotation.behavior.DynamicProperty; import org.apache.nifi.annotation.behavior.EventDriven; import org.apache.nifi.annotation.behavior.SideEffectFree; import org.apache.nifi.annotation.behavior.SupportsBatching; @@ -115,6 +116,12 @@ import org.apache.nifi.processor.util.StandardValidators; + "to be used as part of the hash. If the regular expression contains a capturing group, only the value of the capturing " + "group will be used.") @WritesAttribute(attribute="", description="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 property.") +@DynamicProperty(name="A flowfile attribute key for attribute inspection", description="A " + + "valid regular expression. This regular expression is evaluated against the " + + "flowfile attribute values. If the regular expression contains a capturing " + + "group, the value of that group will be used when comparing flow file " + + "attributes. Otherwise, the original flow file attribute's value will be used " + + "if and only if the value matches the given regular expression.") public class HashAttribute extends AbstractProcessor { public static final PropertyDescriptor HASH_VALUE_ATTRIBUTE = new PropertyDescriptor.Builder() diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/InvokeHTTP.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/InvokeHTTP.java index 9b98623410..dda2950beb 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/InvokeHTTP.java +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/InvokeHTTP.java @@ -49,6 +49,7 @@ import javax.net.ssl.SSLContext; import javax.net.ssl.SSLSession; import org.apache.commons.lang3.StringUtils; +import org.apache.nifi.annotation.behavior.DynamicProperty; import org.apache.nifi.annotation.behavior.SupportsBatching; import org.apache.nifi.annotation.documentation.CapabilityDescription; import org.apache.nifi.annotation.documentation.Tags; @@ -77,6 +78,8 @@ import org.joda.time.format.DateTimeFormatter; @WritesAttribute(attribute = "invokehttp.request.url", description = "The request URL"), @WritesAttribute(attribute = "invokehttp.tx.id", description = "The transaction ID that is returned after reading the response"), @WritesAttribute(attribute = "invokehttp.remote.dn", description = "The DN of the remote server") }) +@DynamicProperty(name="Trusted Hostname)", description="Bypass the normal truststore hostname verifier to allow the specified (single) remote hostname as trusted " + + "Enabling this property has MITM security implications, use wisely. Only valid with SSL (HTTPS) connections.") public final class InvokeHTTP extends AbstractProcessor { //-- properties --// diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PutFTP.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PutFTP.java index 6d80038e50..f8ac6bf3fb 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PutFTP.java +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PutFTP.java @@ -26,6 +26,8 @@ import java.util.concurrent.atomic.AtomicReference; import java.util.regex.Matcher; import java.util.regex.Pattern; +import org.apache.nifi.annotation.behavior.DynamicProperties; +import org.apache.nifi.annotation.behavior.DynamicProperty; import org.apache.nifi.annotation.behavior.SupportsBatching; import org.apache.nifi.annotation.documentation.CapabilityDescription; import org.apache.nifi.annotation.documentation.SeeAlso; @@ -42,6 +44,8 @@ import org.apache.nifi.processors.standard.util.FTPTransfer; @Tags({"remote", "copy", "egress", "put", "ftp", "archive", "files"}) @CapabilityDescription("Sends FlowFiles to an FTP Server") @SeeAlso(GetFTP.class) +@DynamicProperties({@DynamicProperty(name="pre.cmd.*", description="Not used"), + @DynamicProperty(name="post.cmd.*", description="Not used")}) public class PutFTP extends PutFileTransfer { private static final Pattern PRE_SEND_CMD_PATTERN = Pattern.compile("^pre\\.cmd\\.(\\d+)$"); diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PutSFTP.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PutSFTP.java index d3b93e7e87..dffbd14efc 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PutSFTP.java +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PutSFTP.java @@ -20,6 +20,7 @@ import java.util.ArrayList; import java.util.Collections; import java.util.List; +import org.apache.nifi.annotation.behavior.DynamicProperty; import org.apache.nifi.annotation.behavior.SupportsBatching; import org.apache.nifi.annotation.documentation.CapabilityDescription; import org.apache.nifi.annotation.documentation.SeeAlso; @@ -33,6 +34,7 @@ import org.apache.nifi.processors.standard.util.SFTPTransfer; @Tags({"remote", "copy", "egress", "put", "sftp", "archive", "files"}) @CapabilityDescription("Sends FlowFiles to an SFTP Server") @SeeAlso(GetSFTP.class) +@DynamicProperty(name="Disable Directory Listing", description="Disables directory listings before operations which might fail, such as configurations which create directory structures.") public class PutSFTP extends PutFileTransfer { private List properties; diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/RouteOnAttribute.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/RouteOnAttribute.java index 1fe78af956..11d8615aee 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/RouteOnAttribute.java +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/RouteOnAttribute.java @@ -26,6 +26,13 @@ import java.util.Map; import java.util.Set; import java.util.concurrent.atomic.AtomicReference; +import org.apache.nifi.annotation.behavior.DynamicProperty; +import org.apache.nifi.annotation.behavior.EventDriven; +import org.apache.nifi.annotation.behavior.SideEffectFree; +import org.apache.nifi.annotation.behavior.SupportsBatching; +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.documentation.DynamicRelationships; +import org.apache.nifi.annotation.documentation.Tags; import org.apache.nifi.components.AllowableValue; import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.components.PropertyValue; @@ -37,11 +44,6 @@ 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.behavior.EventDriven; -import org.apache.nifi.annotation.behavior.SideEffectFree; -import org.apache.nifi.annotation.behavior.SupportsBatching; -import org.apache.nifi.annotation.documentation.Tags; import org.apache.nifi.processor.util.StandardValidators; /** @@ -65,6 +67,8 @@ import org.apache.nifi.processor.util.StandardValidators; @SupportsBatching @Tags({"attributes", "routing", "Attribute Expression Language", "regexp", "regex", "Regular Expression", "Expression Language"}) @CapabilityDescription("Routes FlowFiles based on their Attributes using the Attribute Expression Language") +@DynamicProperty(name="Relationship", description="Attribute Expression Language", supportsExpressionLanguage=true) +@DynamicRelationships(name="Name from Dynamic Property", description="FlowFiles that match the Dynamic Property's Attribute Expression Language") public class RouteOnAttribute extends AbstractProcessor { public static final String ROUTE_ATTRIBUTE_KEY = "RouteOnAttribute.Route"; diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/RouteOnContent.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/RouteOnContent.java index 3e581d2a26..b95402d4af 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/RouteOnContent.java +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/RouteOnContent.java @@ -41,6 +41,8 @@ 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.DynamicRelationships; +import org.apache.nifi.annotation.behavior.DynamicProperty; import org.apache.nifi.annotation.behavior.EventDriven; import org.apache.nifi.annotation.behavior.SideEffectFree; import org.apache.nifi.annotation.behavior.SupportsBatching; @@ -58,6 +60,8 @@ import org.apache.nifi.util.IntegerHolder; + "of the property is the name of the relationship and the value is a Regular Expression to match against the FlowFile " + "content. User-Defined properties do support the Attribute Expression Language, but the results are interpreted as " + "literal values, not Regular Expressions") +@DynamicProperty(name="Relationship", description="A Regular Expression", supportsExpressionLanguage=true) +@DynamicRelationships(name="Name from Dynamic Property", description="FlowFiles that match the Dynamic Property's Regular Expression") public class RouteOnContent extends AbstractProcessor { public static final String ROUTE_ATTRIBUTE_KEY = "RouteOnContent.Route"; diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/TransformXml.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/TransformXml.java index 8a2feb837d..778fb2902d 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/TransformXml.java +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/TransformXml.java @@ -33,27 +33,28 @@ import javax.xml.transform.TransformerFactory; import javax.xml.transform.stream.StreamResult; import javax.xml.transform.stream.StreamSource; +import org.apache.nifi.annotation.behavior.DynamicProperty; +import org.apache.nifi.annotation.behavior.EventDriven; +import org.apache.nifi.annotation.behavior.SideEffectFree; +import org.apache.nifi.annotation.behavior.SupportsBatching; +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.documentation.Tags; import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.components.ValidationContext; import org.apache.nifi.components.ValidationResult; import org.apache.nifi.components.Validator; import org.apache.nifi.expression.AttributeExpression; import org.apache.nifi.flowfile.FlowFile; -import org.apache.nifi.stream.io.BufferedInputStream; import org.apache.nifi.logging.ProcessorLog; 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.behavior.EventDriven; -import org.apache.nifi.annotation.behavior.SideEffectFree; -import org.apache.nifi.annotation.behavior.SupportsBatching; -import org.apache.nifi.annotation.documentation.Tags; import org.apache.nifi.processor.exception.ProcessException; import org.apache.nifi.processor.io.StreamCallback; import org.apache.nifi.processor.util.StandardValidators; +import org.apache.nifi.stream.io.BufferedInputStream; import org.apache.nifi.util.StopWatch; import org.apache.nifi.util.Tuple; @@ -64,6 +65,7 @@ import org.apache.nifi.util.Tuple; @CapabilityDescription("Applies the provided XSLT file to the flowfile XML payload. A new FlowFile is created " + "with transformed content and is routed to the 'success' relationship. If the XSL transform " + "fails, the original FlowFile is routed to the 'failure' relationship") +@DynamicProperty(name="An XSLT transform parameter name", description="An XSLT transform parameter value", supportsExpressionLanguage=true) public class TransformXml extends AbstractProcessor { public static final PropertyDescriptor XSLT_FILE_NAME = new PropertyDescriptor.Builder() diff --git a/nifi/nifi-nar-bundles/nifi-update-attribute-bundle/nifi-update-attribute-processor/src/main/java/org/apache/nifi/processors/attributes/UpdateAttribute.java b/nifi/nifi-nar-bundles/nifi-update-attribute-bundle/nifi-update-attribute-processor/src/main/java/org/apache/nifi/processors/attributes/UpdateAttribute.java index 5e1e0267d2..1933ab8f0c 100644 --- a/nifi/nifi-nar-bundles/nifi-update-attribute-bundle/nifi-update-attribute-processor/src/main/java/org/apache/nifi/processors/attributes/UpdateAttribute.java +++ b/nifi/nifi-nar-bundles/nifi-update-attribute-bundle/nifi-update-attribute-processor/src/main/java/org/apache/nifi/processors/attributes/UpdateAttribute.java @@ -30,6 +30,7 @@ import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.atomic.AtomicReference; +import org.apache.nifi.annotation.behavior.DynamicProperty; import org.apache.nifi.annotation.behavior.EventDriven; import org.apache.nifi.annotation.behavior.SideEffectFree; import org.apache.nifi.annotation.documentation.CapabilityDescription; @@ -112,6 +113,7 @@ import org.apache.commons.lang3.StringUtils; @SideEffectFree @Tags({"attributes", "modification", "update", "Attribute Expression Language"}) @CapabilityDescription("Updates the Attributes for a FlowFile by using the Attribute Expression Language") +@DynamicProperty(name="A FlowFile attribute to update", description="The value to set it to", supportsExpressionLanguage=true) public class UpdateAttribute extends AbstractProcessor implements Searchable { private final AtomicReference criteriaCache = new AtomicReference<>(null); From fd89849f5f081c4a0c67c0dec63130800ba6f03a Mon Sep 17 00:00:00 2001 From: danbress Date: Wed, 18 Mar 2015 15:30:50 -0400 Subject: [PATCH 053/116] NIFI-309 making sure dynamic property exists --- .../nifi/documentation/html/HtmlDocumentationWriter.java | 4 +++- 1 file changed, 3 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 7def25c712..6fafb050b4 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 @@ -409,7 +409,9 @@ public class HtmlDocumentationWriter implements DocumentationWriter { } final DynamicProperty dynProp = configurableComponent.getClass().getAnnotation(DynamicProperty.class); - dynamicProperties.add(dynProp); + if (dynProp != null) { + dynamicProperties.add(dynProp); + } return dynamicProperties; } From c98ddca2eb10d4b73ba8a7a7d017983e624ea21b Mon Sep 17 00:00:00 2001 From: danbress Date: Wed, 18 Mar 2015 15:52:57 -0400 Subject: [PATCH 054/116] NIFI-309 adding a value to @DynamicProperty this lets you describe the name, the value, and what the two together mean --- .../apache/nifi/annotation/behavior/DynamicProperty.java | 8 +++++++- .../nifi/documentation/html/HtmlDocumentationWriter.java | 2 ++ .../documentation/example/FullyDocumentedProcessor.java | 2 +- .../apache/nifi/processors/standard/DistributeLoad.java | 3 ++- .../apache/nifi/processors/standard/EvaluateJsonPath.java | 3 ++- .../processors/standard/EvaluateRegularExpression.java | 2 +- .../apache/nifi/processors/standard/EvaluateXPath.java | 3 ++- .../apache/nifi/processors/standard/EvaluateXQuery.java | 3 ++- .../apache/nifi/processors/standard/ExecuteProcess.java | 2 +- .../nifi/processors/standard/HandleHttpResponse.java | 2 +- .../apache/nifi/processors/standard/HashAttribute.java | 3 +-- .../org/apache/nifi/processors/standard/InvokeHTTP.java | 2 +- .../java/org/apache/nifi/processors/standard/PutFTP.java | 4 ++-- .../java/org/apache/nifi/processors/standard/PutSFTP.java | 2 +- .../apache/nifi/processors/standard/RouteOnAttribute.java | 3 ++- .../apache/nifi/processors/standard/RouteOnContent.java | 3 ++- .../org/apache/nifi/processors/standard/TransformXml.java | 3 ++- .../nifi/processors/attributes/UpdateAttribute.java | 3 ++- 18 files changed, 34 insertions(+), 19 deletions(-) diff --git a/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/behavior/DynamicProperty.java b/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/behavior/DynamicProperty.java index 7b2150dfef..a07817c1ad 100644 --- a/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/behavior/DynamicProperty.java +++ b/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/behavior/DynamicProperty.java @@ -36,7 +36,13 @@ public @interface DynamicProperty { * language */ public boolean supportsExpressionLanguage() default false; - + + /** + * A description of what the value of the dynamic property may be + * @return a description of what the value of the dynamic property may be + */ + public String value(); + /** * Provides a description of what the meaning of the property is, and what the expected values are * @return a description of what the meaning of the property is, and what the expected values are 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 6fafb050b4..c4261a1516 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 @@ -374,11 +374,13 @@ public class HtmlDocumentationWriter implements DocumentationWriter { xmlStreamWriter.writeStartElement("table"); xmlStreamWriter.writeStartElement("tr"); writeSimpleElement(xmlStreamWriter, "th", "Name"); + writeSimpleElement(xmlStreamWriter, "th", "Value"); writeSimpleElement(xmlStreamWriter, "th", "Description"); xmlStreamWriter.writeEndElement(); for (final DynamicProperty dynamicProperty : dynamicProperties) { xmlStreamWriter.writeStartElement("tr"); writeSimpleElement(xmlStreamWriter, "td", dynamicProperty.name()); + writeSimpleElement(xmlStreamWriter, "td", dynamicProperty.value()); xmlStreamWriter.writeStartElement("td"); xmlStreamWriter.writeCharacters(dynamicProperty.description()); if (dynamicProperty.supportsExpressionLanguage()) { 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 782d30548e..9a5c3983e6 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 @@ -45,7 +45,7 @@ import org.apache.nifi.processor.util.StandardValidators; @WritesAttributes({@WritesAttribute(attribute="first", description="this is the first attribute i write"), @WritesAttribute(attribute="second")}) @ReadsAttribute(attribute = "incoming", description="this specifies the format of the thing") @SeeAlso(value={FullyDocumentedControllerService.class, FullyDocumentedReportingTask.class}, classNames={"org.apache.nifi.processor.ExampleProcessor"}) -@DynamicProperty(name="the relationship to route to", supportsExpressionLanguage=true, description="some XPath") +@DynamicProperty(name="Relationship Name", supportsExpressionLanguage=true, value="some XPath", description="Routes FlowFiles to relationships based on XPath") @DynamicRelationships(name="name from dynamic property", description="all files that match the properties XPath") public class FullyDocumentedProcessor extends AbstractProcessor { 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 a0dcc79072..f053ce3473 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 @@ -63,7 +63,8 @@ import org.apache.nifi.processor.util.StandardValidators; + "strategy, the default is to assign each destination a weighting of 1 (evenly distributed). However, optional properties" + "can be added to the change this; adding a property with the name '5' and value '10' means that the relationship with name " + "'5' will be receive 10 FlowFiles in each iteration instead of 1.") -@DynamicProperty(name="The relationship name(positive number)", description="The number of FlowFiles to this relationship per iteration") +@DynamicProperty(name="The relationship name(positive number)", value="The relationship Weight(positive number)", description="adding a property with the name '5' and value '10' means that the relationship with name " + + "'5' will be receive 10 FlowFiles in each iteration instead of 1.") @DynamicRelationships(name="A number 1..", description="FlowFiles are sent to this relationship per the ") public class DistributeLoad extends AbstractProcessor { diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateJsonPath.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateJsonPath.java index e6356f0bfd..c29a4d2960 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateJsonPath.java +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateJsonPath.java @@ -66,7 +66,8 @@ import java.util.concurrent.ConcurrentMap; + "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.'") -@DynamicProperty(name="A FlowFile attribute(if is set to 'flowfile-attribute')", description="JsonPath expression") +@DynamicProperty(name="A FlowFile attribute(if is set to 'flowfile-attribute')", value="A JsonPath expression", description="If ='flowfile-attribute' then that FlowFile attribute " + + "will be set to any JSON objects that match the JsonPath. If ='flowfile-content' then the FlowFile content will be updated to any JSON objects that match the JsonPath.") public class EvaluateJsonPath extends AbstractJsonPathProcessor { public static final String DESTINATION_ATTRIBUTE = "flowfile-attribute"; diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateRegularExpression.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateRegularExpression.java index e5598d4c4e..35436ca9c7 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateRegularExpression.java +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateRegularExpression.java @@ -64,7 +64,7 @@ import org.apache.commons.lang3.StringUtils; + "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.") @Deprecated -@DynamicProperty(name="A FlowFile attribute", description="A regular expression with exactly one capturing group") +@DynamicProperty(name="A FlowFile attribute", value="A regular expression with exactly one capturing group", description="Will update the specified FlowFile attribute with the group captured by the regular expression") public class EvaluateRegularExpression extends AbstractProcessor { public static final PropertyDescriptor CHARACTER_SET = new PropertyDescriptor.Builder() diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateXPath.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateXPath.java index 5db3b0b864..7852364202 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateXPath.java +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateXPath.java @@ -94,7 +94,8 @@ import org.xml.sax.InputSource; + "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'") @WritesAttribute(attribute="user-defined", description="This processor adds user-defined attributes if the property is set to flowfile-attribute.") -@DynamicProperty(name="A FlowFile attribute(if is set to 'flowfile-attribute'", description="An XPath expression") +@DynamicProperty(name="A FlowFile attribute(if is set to 'flowfile-attribute'", value="An XPath expression", description="If ='flowfile-attribute' " + +"then the FlowFile attribute is set to the result of the XPath Expression. If ='flowfile-content' then the FlowFile content is set to the result of the XPath Expression.") public class EvaluateXPath extends AbstractProcessor { public static final String DESTINATION_ATTRIBUTE = "flowfile-attribute"; diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateXQuery.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateXQuery.java index 143c6b4d5f..8bc1f9feb5 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateXQuery.java +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateXQuery.java @@ -95,7 +95,8 @@ import org.xml.sax.InputSource; + "Destination is 'flowfile-attribute' and the XQueries matche nothing, no attributes will be applied to the " + "FlowFile.") @WritesAttribute(attribute="user-defined", description="This processor adds user-defined attributes if the property is set to flowfile-attribute .") -@DynamicProperty(name="A FlowFile attribute(if is set to 'flowfile-attribute'", description="An XQuery") +@DynamicProperty(name="A FlowFile attribute(if is set to 'flowfile-attribute'", value="An XQuery", description="If ='flowfile-attribute' " + + "then the FlowFile attribute is set to the result of the XQuery. If ='flowfile-content' then the FlowFile content is set to the result of the XQuery.") public class EvaluateXQuery extends AbstractProcessor { public static final String DESTINATION_ATTRIBUTE = "flowfile-attribute"; diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ExecuteProcess.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ExecuteProcess.java index 319cb2cd96..61cf317ae6 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ExecuteProcess.java +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ExecuteProcess.java @@ -62,7 +62,7 @@ import org.apache.nifi.processor.util.StandardValidators; @CapabilityDescription("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.") -@DynamicProperty(name="An environment variable name", description="An environment variable value") +@DynamicProperty(name="An environment variable name", value="An environment variable value", description="These environment variables are passed to the process spawned by this Processor") public class ExecuteProcess extends AbstractProcessor { public static final PropertyDescriptor COMMAND = new PropertyDescriptor.Builder() diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/HandleHttpResponse.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/HandleHttpResponse.java index a83b4ce169..9c7cfaf36f 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/HandleHttpResponse.java +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/HandleHttpResponse.java @@ -41,7 +41,7 @@ import org.apache.nifi.processor.util.StandardValidators; @Tags({"http", "https", "response", "egress", "web service"}) @CapabilityDescription("Sends an HTTP Response to the Requestor that generated a FlowFile. This Processor is designed to be used in conjunction with the HandleHttpRequest in order to create a web service.") -@DynamicProperty(name="An HTTP header name", description="An HTTP header value") +@DynamicProperty(name="An HTTP header name", value="An HTTP header value", description="These HTTPHeaders are set in the HTTP Response") public class HandleHttpResponse extends AbstractProcessor { public static final Pattern NUMBER_PATTERN = Pattern.compile("[0-9]+"); public static final String HTTP_CONTEXT_ID = "http.context.identifier"; diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/HashAttribute.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/HashAttribute.java index 9c5c0b54d2..816f98cd24 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/HashAttribute.java +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/HashAttribute.java @@ -116,8 +116,7 @@ import org.apache.nifi.processor.util.StandardValidators; + "to be used as part of the hash. If the regular expression contains a capturing group, only the value of the capturing " + "group will be used.") @WritesAttribute(attribute="", description="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 property.") -@DynamicProperty(name="A flowfile attribute key for attribute inspection", description="A " + - "valid regular expression. This regular expression is evaluated against the " + +@DynamicProperty(name="A flowfile attribute key for attribute inspection", value="A Regular Expression", description="This regular expression is evaluated against the " + "flowfile attribute values. If the regular expression contains a capturing " + "group, the value of that group will be used when comparing flow file " + "attributes. Otherwise, the original flow file attribute's value will be used " + diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/InvokeHTTP.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/InvokeHTTP.java index dda2950beb..a2bab1d5d4 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/InvokeHTTP.java +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/InvokeHTTP.java @@ -78,7 +78,7 @@ import org.joda.time.format.DateTimeFormatter; @WritesAttribute(attribute = "invokehttp.request.url", description = "The request URL"), @WritesAttribute(attribute = "invokehttp.tx.id", description = "The transaction ID that is returned after reading the response"), @WritesAttribute(attribute = "invokehttp.remote.dn", description = "The DN of the remote server") }) -@DynamicProperty(name="Trusted Hostname)", description="Bypass the normal truststore hostname verifier to allow the specified (single) remote hostname as trusted " +@DynamicProperty(name="Trusted Hostname", value="A hostname", description="Bypass the normal truststore hostname verifier to allow the specified (single) remote hostname as trusted " + "Enabling this property has MITM security implications, use wisely. Only valid with SSL (HTTPS) connections.") public final class InvokeHTTP extends AbstractProcessor { diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PutFTP.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PutFTP.java index f8ac6bf3fb..e8fec4d555 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PutFTP.java +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PutFTP.java @@ -44,8 +44,8 @@ import org.apache.nifi.processors.standard.util.FTPTransfer; @Tags({"remote", "copy", "egress", "put", "ftp", "archive", "files"}) @CapabilityDescription("Sends FlowFiles to an FTP Server") @SeeAlso(GetFTP.class) -@DynamicProperties({@DynamicProperty(name="pre.cmd.*", description="Not used"), - @DynamicProperty(name="post.cmd.*", description="Not used")}) +@DynamicProperties({@DynamicProperty(name="pre.cmd._____", value="Not used", description="The command specified in the key will be executed before doing a put"), + @DynamicProperty(name="post.cmd._____", value="Not used", description="The command specified in the key will be executed after doing a put")}) public class PutFTP extends PutFileTransfer { private static final Pattern PRE_SEND_CMD_PATTERN = Pattern.compile("^pre\\.cmd\\.(\\d+)$"); diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PutSFTP.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PutSFTP.java index dffbd14efc..300878fcf1 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PutSFTP.java +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PutSFTP.java @@ -34,7 +34,7 @@ import org.apache.nifi.processors.standard.util.SFTPTransfer; @Tags({"remote", "copy", "egress", "put", "sftp", "archive", "files"}) @CapabilityDescription("Sends FlowFiles to an SFTP Server") @SeeAlso(GetSFTP.class) -@DynamicProperty(name="Disable Directory Listing", description="Disables directory listings before operations which might fail, such as configurations which create directory structures.") +@DynamicProperty(name="Disable Directory Listing", value="true or false", description="Disables directory listings before operations which might fail, such as configurations which create directory structures.") public class PutSFTP extends PutFileTransfer { private List properties; diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/RouteOnAttribute.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/RouteOnAttribute.java index 11d8615aee..caab595ad8 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/RouteOnAttribute.java +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/RouteOnAttribute.java @@ -67,7 +67,8 @@ import org.apache.nifi.processor.util.StandardValidators; @SupportsBatching @Tags({"attributes", "routing", "Attribute Expression Language", "regexp", "regex", "Regular Expression", "Expression Language"}) @CapabilityDescription("Routes FlowFiles based on their Attributes using the Attribute Expression Language") -@DynamicProperty(name="Relationship", description="Attribute Expression Language", supportsExpressionLanguage=true) +@DynamicProperty(name="Relationship Name", value="Attribute Expression Language", supportsExpressionLanguage=true, description="Routes FlowFiles whose " + +"attributes match the Attribute Expression Language specified in the Dynamic Property Value to the Relationship specified in the Dynamic Property Key") @DynamicRelationships(name="Name from Dynamic Property", description="FlowFiles that match the Dynamic Property's Attribute Expression Language") public class RouteOnAttribute extends AbstractProcessor { diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/RouteOnContent.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/RouteOnContent.java index b95402d4af..a7b1fc4b86 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/RouteOnContent.java +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/RouteOnContent.java @@ -60,7 +60,8 @@ import org.apache.nifi.util.IntegerHolder; + "of the property is the name of the relationship and the value is a Regular Expression to match against the FlowFile " + "content. User-Defined properties do support the Attribute Expression Language, but the results are interpreted as " + "literal values, not Regular Expressions") -@DynamicProperty(name="Relationship", description="A Regular Expression", supportsExpressionLanguage=true) +@DynamicProperty(name="Relationship Name", value="A Regular Expression", supportsExpressionLanguage=true, description="Routes FlowFiles whose " + + "content matches the regular expressoin defined by Dynamic Property's value to the Relationship defined by the Dynamic Property's key") @DynamicRelationships(name="Name from Dynamic Property", description="FlowFiles that match the Dynamic Property's Regular Expression") public class RouteOnContent extends AbstractProcessor { diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/TransformXml.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/TransformXml.java index 778fb2902d..ff3790272d 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/TransformXml.java +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/TransformXml.java @@ -65,7 +65,8 @@ import org.apache.nifi.util.Tuple; @CapabilityDescription("Applies the provided XSLT file to the flowfile XML payload. A new FlowFile is created " + "with transformed content and is routed to the 'success' relationship. If the XSL transform " + "fails, the original FlowFile is routed to the 'failure' relationship") -@DynamicProperty(name="An XSLT transform parameter name", description="An XSLT transform parameter value", supportsExpressionLanguage=true) +@DynamicProperty(name="An XSLT transform parameter name", value="An XSLT transform parameter value", supportsExpressionLanguage=true, +description="These XSLT parameters are passed to the transformer") public class TransformXml extends AbstractProcessor { public static final PropertyDescriptor XSLT_FILE_NAME = new PropertyDescriptor.Builder() diff --git a/nifi/nifi-nar-bundles/nifi-update-attribute-bundle/nifi-update-attribute-processor/src/main/java/org/apache/nifi/processors/attributes/UpdateAttribute.java b/nifi/nifi-nar-bundles/nifi-update-attribute-bundle/nifi-update-attribute-processor/src/main/java/org/apache/nifi/processors/attributes/UpdateAttribute.java index 1933ab8f0c..afdc4dbfbe 100644 --- a/nifi/nifi-nar-bundles/nifi-update-attribute-bundle/nifi-update-attribute-processor/src/main/java/org/apache/nifi/processors/attributes/UpdateAttribute.java +++ b/nifi/nifi-nar-bundles/nifi-update-attribute-bundle/nifi-update-attribute-processor/src/main/java/org/apache/nifi/processors/attributes/UpdateAttribute.java @@ -113,7 +113,8 @@ import org.apache.commons.lang3.StringUtils; @SideEffectFree @Tags({"attributes", "modification", "update", "Attribute Expression Language"}) @CapabilityDescription("Updates the Attributes for a FlowFile by using the Attribute Expression Language") -@DynamicProperty(name="A FlowFile attribute to update", description="The value to set it to", supportsExpressionLanguage=true) +@DynamicProperty(name="A FlowFile attribute to update", value="The value to set it to", supportsExpressionLanguage=true, description="Updates a " + +"FlowFile attribute specified by the Dynamic Property's key with the value specified by the Dynamic Property's value") public class UpdateAttribute extends AbstractProcessor implements Searchable { private final AtomicReference criteriaCache = new AtomicReference<>(null); From c441f9160dc3e0cc686c7e73109966120666ada6 Mon Sep 17 00:00:00 2001 From: danbress Date: Wed, 18 Mar 2015 16:00:55 -0400 Subject: [PATCH 055/116] NIFI-309 - cleaning up additionalDetails.html --- .../nifi/processors/standard/PutFTP.java | 12 ++++- .../additionalDetails.html | 40 ---------------- .../additionalDetails.html | 27 ----------- .../additionalDetails.html | 4 -- .../additionalDetails.html | 48 ------------------- 5 files changed, 10 insertions(+), 121 deletions(-) delete mode 100644 nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.DistributeLoad/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/additionalDetails.html diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PutFTP.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PutFTP.java index e8fec4d555..6e7566181c 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PutFTP.java +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PutFTP.java @@ -44,8 +44,16 @@ import org.apache.nifi.processors.standard.util.FTPTransfer; @Tags({"remote", "copy", "egress", "put", "ftp", "archive", "files"}) @CapabilityDescription("Sends FlowFiles to an FTP Server") @SeeAlso(GetFTP.class) -@DynamicProperties({@DynamicProperty(name="pre.cmd._____", value="Not used", description="The command specified in the key will be executed before doing a put"), - @DynamicProperty(name="post.cmd._____", value="Not used", description="The command specified in the key will be executed after doing a put")}) +@DynamicProperties({@DynamicProperty(name="pre.cmd._____", value="Not used", description="The command specified in the key will be executed before doing a put. 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. While this is the same as sending a quote command, it is very important that" + + " you leave off the ."), + @DynamicProperty(name="post.cmd._____", value="Not used", description="The command specified in the key will be executed after doing a put. 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. While this is the same as sending a quote command, it is very important that" + + " you leave off the .")}) public class PutFTP extends PutFileTransfer { private static final Pattern PRE_SEND_CMD_PATTERN = Pattern.compile("^pre\\.cmd\\.(\\d+)$"); 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 deleted file mode 100644 index f593bed4b4..0000000000 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.DistributeLoad/additionalDetails.html +++ /dev/null @@ -1,40 +0,0 @@ - - - - - - DistributeLoad - - - - - - 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 index c1d80aa43d..59bc7070d3 100644 --- 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 @@ -27,32 +27,5 @@ 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.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 b5a31fc073..ec3179a4d7 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,10 +22,6 @@ -

    - Modifies Attributes: -

    -

    The following MIME Types are detected:

      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 deleted file mode 100644 index d8afa01da0..0000000000 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.PutFTP/additionalDetails.html +++ /dev/null @@ -1,48 +0,0 @@ - - - - - - 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 . - (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
        • -
        -
      • -
      - - From 86a429ff767527c08505c6cdf88d64b87f6fc1cc Mon Sep 17 00:00:00 2001 From: danbress Date: Wed, 18 Mar 2015 16:20:05 -0400 Subject: [PATCH 056/116] NIFI-309 renaming DynamicRelationships --- .../{DynamicRelationships.java => DynamicRelationship.java} | 0 1 file changed, 0 insertions(+), 0 deletions(-) rename nifi/nifi-api/src/main/java/org/apache/nifi/annotation/documentation/{DynamicRelationships.java => DynamicRelationship.java} (100%) diff --git a/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/documentation/DynamicRelationships.java b/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/documentation/DynamicRelationship.java similarity index 100% rename from nifi/nifi-api/src/main/java/org/apache/nifi/annotation/documentation/DynamicRelationships.java rename to nifi/nifi-api/src/main/java/org/apache/nifi/annotation/documentation/DynamicRelationship.java From a19215fc8deab582015ace40e41db8095e1f74f1 Mon Sep 17 00:00:00 2001 From: danbress Date: Wed, 18 Mar 2015 16:20:38 -0400 Subject: [PATCH 057/116] NIFI-309 applying ASF License --- .../annotation/behavior/DynamicProperties.java | 16 ++++++++++++++++ .../annotation/behavior/DynamicProperty.java | 16 ++++++++++++++++ .../documentation/DynamicRelationship.java | 18 +++++++++++++++++- 3 files changed, 49 insertions(+), 1 deletion(-) diff --git a/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/behavior/DynamicProperties.java b/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/behavior/DynamicProperties.java index 4fdfd080b3..6dbc86d792 100644 --- a/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/behavior/DynamicProperties.java +++ b/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/behavior/DynamicProperties.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.annotation.behavior; import java.lang.annotation.Documented; diff --git a/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/behavior/DynamicProperty.java b/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/behavior/DynamicProperty.java index a07817c1ad..07f349c388 100644 --- a/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/behavior/DynamicProperty.java +++ b/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/behavior/DynamicProperty.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.annotation.behavior; import java.lang.annotation.Documented; diff --git a/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/documentation/DynamicRelationship.java b/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/documentation/DynamicRelationship.java index 0047cb13b8..e656bd3fc4 100644 --- a/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/documentation/DynamicRelationship.java +++ b/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/documentation/DynamicRelationship.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.annotation.documentation; import java.lang.annotation.Documented; @@ -23,7 +39,7 @@ import org.apache.nifi.processor.Relationship; @Target({ ElementType.TYPE }) @Retention(RetentionPolicy.RUNTIME) @Inherited -public @interface DynamicRelationships { +public @interface DynamicRelationship { /** * Describes the name(s) of the dynamic relationship(s) * From 5232ba33a21f04f05038837c0973e266793eef87 Mon Sep 17 00:00:00 2001 From: danbress Date: Wed, 18 Mar 2015 16:20:57 -0400 Subject: [PATCH 058/116] NIFI-309 renaming DynamicRelationships --- .../html/HtmlProcessorDocumentationWriter.java | 12 ++++++------ .../example/FullyDocumentedProcessor.java | 4 ++-- .../nifi/processors/standard/DistributeLoad.java | 4 ++-- .../nifi/processors/standard/EvaluateJsonPath.java | 2 +- .../nifi/processors/standard/RouteOnAttribute.java | 4 ++-- .../nifi/processors/standard/RouteOnContent.java | 4 ++-- 6 files changed, 15 insertions(+), 15 deletions(-) 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 6eed4dcbf5..f486a50409 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 @@ -23,7 +23,7 @@ import javax.xml.stream.XMLStreamException; import javax.xml.stream.XMLStreamWriter; import org.apache.commons.lang3.StringUtils; -import org.apache.nifi.annotation.documentation.DynamicRelationships; +import org.apache.nifi.annotation.documentation.DynamicRelationship; import org.apache.nifi.annotation.documentation.ReadsAttribute; import org.apache.nifi.annotation.documentation.ReadsAttributes; import org.apache.nifi.annotation.documentation.WritesAttribute; @@ -232,7 +232,7 @@ public class HtmlProcessorDocumentationWriter extends HtmlDocumentationWriter { */ private void writeDynamicRelationships(final Processor processor, final XMLStreamWriter xmlStreamWriter) throws XMLStreamException { - List dynamicRelationships = getDynamicRelationships(processor); + List dynamicRelationships = getDynamicRelationships(processor); if (dynamicRelationships.size() > 0) { writeSimpleElement(xmlStreamWriter, "h3", "Dynamic Relationships: "); @@ -242,7 +242,7 @@ public class HtmlProcessorDocumentationWriter extends HtmlDocumentationWriter { writeSimpleElement(xmlStreamWriter, "th", "Description"); xmlStreamWriter.writeEndElement(); - for (DynamicRelationships dynamicRelationship : dynamicRelationships) { + for (DynamicRelationship dynamicRelationship : dynamicRelationships) { xmlStreamWriter.writeStartElement("tr"); writeSimpleElement(xmlStreamWriter, "td", dynamicRelationship.name()); writeSimpleElement(xmlStreamWriter, "td", dynamicRelationship.description()); @@ -252,10 +252,10 @@ public class HtmlProcessorDocumentationWriter extends HtmlDocumentationWriter { } } - private List getDynamicRelationships(Processor processor) { - List results = new ArrayList<>(); + private List getDynamicRelationships(Processor processor) { + List results = new ArrayList<>(); - DynamicRelationships dynamicRelationships = processor.getClass().getAnnotation(DynamicRelationships.class); + DynamicRelationship dynamicRelationships = processor.getClass().getAnnotation(DynamicRelationship.class); if (dynamicRelationships != null) { results.add(dynamicRelationships); } 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 9a5c3983e6..dd623adff1 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 @@ -24,7 +24,7 @@ import java.util.Set; import org.apache.nifi.annotation.behavior.DynamicProperty; import org.apache.nifi.annotation.documentation.CapabilityDescription; -import org.apache.nifi.annotation.documentation.DynamicRelationships; +import org.apache.nifi.annotation.documentation.DynamicRelationship; import org.apache.nifi.annotation.documentation.ReadsAttribute; import org.apache.nifi.annotation.documentation.SeeAlso; import org.apache.nifi.annotation.documentation.Tags; @@ -46,7 +46,7 @@ import org.apache.nifi.processor.util.StandardValidators; @ReadsAttribute(attribute = "incoming", description="this specifies the format of the thing") @SeeAlso(value={FullyDocumentedControllerService.class, FullyDocumentedReportingTask.class}, classNames={"org.apache.nifi.processor.ExampleProcessor"}) @DynamicProperty(name="Relationship Name", supportsExpressionLanguage=true, value="some XPath", description="Routes FlowFiles to relationships based on XPath") -@DynamicRelationships(name="name from dynamic property", description="all files that match the properties XPath") +@DynamicRelationship(name="name from dynamic property", description="all files that match the properties XPath") public class FullyDocumentedProcessor extends AbstractProcessor { public static final PropertyDescriptor DIRECTORY = new PropertyDescriptor.Builder().name("Input Directory") 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 f053ce3473..07b897083b 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 @@ -36,7 +36,7 @@ import org.apache.nifi.annotation.behavior.SideEffectFree; import org.apache.nifi.annotation.behavior.SupportsBatching; import org.apache.nifi.annotation.behavior.TriggerWhenAnyDestinationAvailable; import org.apache.nifi.annotation.documentation.CapabilityDescription; -import org.apache.nifi.annotation.documentation.DynamicRelationships; +import org.apache.nifi.annotation.documentation.DynamicRelationship; import org.apache.nifi.annotation.documentation.Tags; import org.apache.nifi.annotation.lifecycle.OnScheduled; import org.apache.nifi.components.PropertyDescriptor; @@ -65,7 +65,7 @@ import org.apache.nifi.processor.util.StandardValidators; + "'5' will be receive 10 FlowFiles in each iteration instead of 1.") @DynamicProperty(name="The relationship name(positive number)", value="The relationship Weight(positive number)", description="adding a property with the name '5' and value '10' means that the relationship with name " + "'5' will be receive 10 FlowFiles in each iteration instead of 1.") -@DynamicRelationships(name="A number 1..", description="FlowFiles are sent to this relationship per the ") +@DynamicRelationship(name="A number 1..", description="FlowFiles are sent to this relationship per the ") public class DistributeLoad extends AbstractProcessor { public static final String STRATEGY_ROUND_ROBIN = "round robin"; diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateJsonPath.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateJsonPath.java index c29a4d2960..d9b21423ad 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateJsonPath.java +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateJsonPath.java @@ -27,7 +27,7 @@ import org.apache.nifi.annotation.behavior.EventDriven; import org.apache.nifi.annotation.behavior.SideEffectFree; import org.apache.nifi.annotation.behavior.SupportsBatching; import org.apache.nifi.annotation.documentation.CapabilityDescription; -import org.apache.nifi.annotation.documentation.DynamicRelationships; +import org.apache.nifi.annotation.documentation.DynamicRelationship; import org.apache.nifi.annotation.documentation.Tags; import org.apache.nifi.annotation.lifecycle.OnRemoved; import org.apache.nifi.components.PropertyDescriptor; diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/RouteOnAttribute.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/RouteOnAttribute.java index caab595ad8..82798b8cba 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/RouteOnAttribute.java +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/RouteOnAttribute.java @@ -31,7 +31,7 @@ import org.apache.nifi.annotation.behavior.EventDriven; import org.apache.nifi.annotation.behavior.SideEffectFree; import org.apache.nifi.annotation.behavior.SupportsBatching; import org.apache.nifi.annotation.documentation.CapabilityDescription; -import org.apache.nifi.annotation.documentation.DynamicRelationships; +import org.apache.nifi.annotation.documentation.DynamicRelationship; import org.apache.nifi.annotation.documentation.Tags; import org.apache.nifi.components.AllowableValue; import org.apache.nifi.components.PropertyDescriptor; @@ -69,7 +69,7 @@ import org.apache.nifi.processor.util.StandardValidators; @CapabilityDescription("Routes FlowFiles based on their Attributes using the Attribute Expression Language") @DynamicProperty(name="Relationship Name", value="Attribute Expression Language", supportsExpressionLanguage=true, description="Routes FlowFiles whose " + "attributes match the Attribute Expression Language specified in the Dynamic Property Value to the Relationship specified in the Dynamic Property Key") -@DynamicRelationships(name="Name from Dynamic Property", description="FlowFiles that match the Dynamic Property's Attribute Expression Language") +@DynamicRelationship(name="Name from Dynamic Property", description="FlowFiles that match the Dynamic Property's Attribute Expression Language") public class RouteOnAttribute extends AbstractProcessor { public static final String ROUTE_ATTRIBUTE_KEY = "RouteOnAttribute.Route"; diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/RouteOnContent.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/RouteOnContent.java index a7b1fc4b86..34adfbb8e7 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/RouteOnContent.java +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/RouteOnContent.java @@ -41,7 +41,7 @@ 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.DynamicRelationships; +import org.apache.nifi.annotation.documentation.DynamicRelationship; import org.apache.nifi.annotation.behavior.DynamicProperty; import org.apache.nifi.annotation.behavior.EventDriven; import org.apache.nifi.annotation.behavior.SideEffectFree; @@ -62,7 +62,7 @@ import org.apache.nifi.util.IntegerHolder; + "literal values, not Regular Expressions") @DynamicProperty(name="Relationship Name", value="A Regular Expression", supportsExpressionLanguage=true, description="Routes FlowFiles whose " + "content matches the regular expressoin defined by Dynamic Property's value to the Relationship defined by the Dynamic Property's key") -@DynamicRelationships(name="Name from Dynamic Property", description="FlowFiles that match the Dynamic Property's Regular Expression") +@DynamicRelationship(name="Name from Dynamic Property", description="FlowFiles that match the Dynamic Property's Regular Expression") public class RouteOnContent extends AbstractProcessor { public static final String ROUTE_ATTRIBUTE_KEY = "RouteOnContent.Route"; From 3b33d41cc3c98ea3d48afe28e4f5b679c2a8e6e7 Mon Sep 17 00:00:00 2001 From: danbress Date: Wed, 18 Mar 2015 16:37:11 -0400 Subject: [PATCH 059/116] NIFI-309 cleaning up HandleHttpRequest/HandleHttpResponse docs --- .../standard/HandleHttpRequest.java | 19 ++++ .../standard/HandleHttpResponse.java | 4 + .../additionalDetails.html | 86 +------------------ .../additionalDetails.html | 13 --- 4 files changed, 24 insertions(+), 98 deletions(-) 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 4386100515..37c504ad2f 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 @@ -45,7 +45,10 @@ import javax.servlet.http.HttpServletResponse; import org.apache.commons.lang3.StringUtils; import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.documentation.SeeAlso; import org.apache.nifi.annotation.documentation.Tags; +import org.apache.nifi.annotation.documentation.WritesAttribute; +import org.apache.nifi.annotation.documentation.WritesAttributes; import org.apache.nifi.annotation.lifecycle.OnScheduled; import org.apache.nifi.annotation.lifecycle.OnStopped; import org.apache.nifi.components.AllowableValue; @@ -74,6 +77,22 @@ import com.sun.jersey.api.client.ClientResponse.Status; @Tags({"http", "https", "request", "listen", "ingress", "web service"}) @CapabilityDescription("Starts an HTTP Server and listens for HTTP Requests. For each request, creates a FlowFile and transfers to 'success'. This Processor is designed to be used in conjunction with the HandleHttpResponse Processor in order to create a Web Service") +@WritesAttributes({@WritesAttribute(attribute = "http.context.identifier", description="An identifier that allows the HandleHttpRequest and HandleHttpResponse to coordinate which FlowFile belongs to which HTTP Request/Response."), + @WritesAttribute(attribute = "mime.type", description="The MIME Type of the data, according to the HTTP Header \"Content-Type\""), + @WritesAttribute(attribute = "http.servlet.path", description="The part of the request URL that is considered the Servlet Path"), + @WritesAttribute(attribute = "http.context.path", description="The part of the request URL that is considered to be the Context Path"), + @WritesAttribute(attribute = "http.method", description="The HTTP Method that was used for the request, such as GET or POST"), + @WritesAttribute(attribute = "http.query.string", description="The query string portion of hte Request URL"), + @WritesAttribute(attribute = "http.remote.host", description="The hostname of the requestor"), + @WritesAttribute(attribute = "http.remote.addr", description="The hostname:port combination of the requestor"), + @WritesAttribute(attribute = "http.remote.user", description="The username of the requestor"), + @WritesAttribute(attribute = "http.request.uri", description="The full Request URL"), + @WritesAttribute(attribute = "http.auth.type", description="The type of HTTP Authorization used"), + @WritesAttribute(attribute = "http.principal.name", description="The name of the authenticated user making the request"), + @WritesAttribute(attribute = "http.subject.dn", description="The Distinguished Name of the requestor. This value will not be populated unless the Processor is configured to use an SSLContext Service"), + @WritesAttribute(attribute = "http.issuer.dn", description="The 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"), + @WritesAttribute(attribute = "http.headers.XXX", description="Each 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\"")}) +@SeeAlso(value={HandleHttpResponse.class}, classNames={"org.apache.nifi.http.StandardHttpContextMap", "org.apache.nifi.ssl.StandardSSLContextService"}) public class HandleHttpRequest extends AbstractProcessor { public static final String HTTP_CONTEXT_ID = "http.context.identifier"; private static final Pattern URL_QUERY_PARAM_DELIMITER = Pattern.compile("&"); diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/HandleHttpResponse.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/HandleHttpResponse.java index 9c7cfaf36f..cd0aebc5df 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/HandleHttpResponse.java +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/HandleHttpResponse.java @@ -28,6 +28,8 @@ import javax.servlet.http.HttpServletResponse; import org.apache.nifi.annotation.behavior.DynamicProperty; import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.documentation.ReadsAttribute; +import org.apache.nifi.annotation.documentation.SeeAlso; import org.apache.nifi.annotation.documentation.Tags; import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.flowfile.FlowFile; @@ -42,6 +44,8 @@ import org.apache.nifi.processor.util.StandardValidators; @Tags({"http", "https", "response", "egress", "web service"}) @CapabilityDescription("Sends an HTTP Response to the Requestor that generated a FlowFile. This Processor is designed to be used in conjunction with the HandleHttpRequest in order to create a web service.") @DynamicProperty(name="An HTTP header name", value="An HTTP header value", description="These HTTPHeaders are set in the HTTP Response") +@ReadsAttribute(attribute="http.context.identifier", description="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.'") +@SeeAlso(value={HandleHttpRequest.class}, classNames={"org.apache.nifi.http.StandardHttpContextMap", "org.apache.nifi.ssl.StandardSSLContextService"}) public class HandleHttpResponse extends AbstractProcessor { public static final Pattern NUMBER_PATTERN = Pattern.compile("[0-9]+"); public static final String HTTP_CONTEXT_ID = "http.context.identifier"; 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 index 17378fb3cc..72aa1e1f1f 100644 --- 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 @@ -37,91 +37,7 @@

      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: + supported, and SSL configuration.

      - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
      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.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 index 5d8993e6c7..1924101427 100644 --- 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 @@ -40,18 +40,5 @@ 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
      -

      - From be78efc010075ecc91d50fdd8f4cd22e78079d80 Mon Sep 17 00:00:00 2001 From: danbress Date: Wed, 18 Mar 2015 16:55:28 -0400 Subject: [PATCH 060/116] NIFI-309 removing print statement --- .../documentation/html/ProcessorDocumentationWriterTest.java | 1 - 1 file changed, 1 deletion(-) 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 ffd31d8b9f..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 @@ -69,7 +69,6 @@ public class ProcessorDocumentationWriterTest { assertNotContains(results, "No description provided."); assertNotContains(results, "No Tags provided."); assertNotContains(results, "Additional Details..."); - System.out.println(results); } @Test From 2c0eb36af8ef7a675b79edee15adfc70f450427f Mon Sep 17 00:00:00 2001 From: danbress Date: Fri, 20 Mar 2015 10:31:07 -0400 Subject: [PATCH 061/116] NIFI-309 adding @DynamicProperty to ExtractText --- .../org/apache/nifi/processors/standard/ExtractText.java | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ExtractText.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ExtractText.java index aa1b7a95b0..caf1cdad86 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ExtractText.java +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ExtractText.java @@ -41,13 +41,13 @@ 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.behavior.DynamicProperty; import org.apache.nifi.annotation.behavior.EventDriven; import org.apache.nifi.annotation.behavior.SideEffectFree; import org.apache.nifi.annotation.behavior.SupportsBatching; import org.apache.nifi.annotation.documentation.Tags; import org.apache.nifi.processor.io.InputStreamCallback; import org.apache.nifi.processor.util.StandardValidators; - import org.apache.nifi.annotation.lifecycle.OnScheduled; @EventDriven @@ -67,7 +67,10 @@ import org.apache.nifi.annotation.lifecycle.OnScheduled; + "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.") - +@DynamicProperty(name="A FlowFile attribute", value="A Regular Expression with one or more capturing group", + description="The first capture group, if any found, will be placed into that attribute name." + + "But all catpure groups, including the matching string sequence itself will also be " + + "provided at that attribute name with an index value provided.") public class ExtractText extends AbstractProcessor { public static final PropertyDescriptor CHARACTER_SET = new PropertyDescriptor.Builder() From 204112d3d2c6dfb6efffd9c75004e0d456926697 Mon Sep 17 00:00:00 2001 From: danbress Date: Fri, 20 Mar 2015 10:37:13 -0400 Subject: [PATCH 062/116] NIFI-309 describing dynamic relationships --- .../documentation/html/HtmlProcessorDocumentationWriter.java | 3 +++ 1 file changed, 3 insertions(+) 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 f486a50409..205e2d17a4 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 @@ -236,6 +236,8 @@ public class HtmlProcessorDocumentationWriter extends HtmlDocumentationWriter { if (dynamicRelationships.size() > 0) { writeSimpleElement(xmlStreamWriter, "h3", "Dynamic Relationships: "); + xmlStreamWriter.writeStartElement("p"); + xmlStreamWriter.writeCharacters("A Dynamic Relationship may be created based on how the user configures the Processor."); xmlStreamWriter.writeStartElement("table"); xmlStreamWriter.writeStartElement("tr"); writeSimpleElement(xmlStreamWriter, "th", "Name"); @@ -249,6 +251,7 @@ public class HtmlProcessorDocumentationWriter extends HtmlDocumentationWriter { xmlStreamWriter.writeEndElement(); } xmlStreamWriter.writeEndElement(); + xmlStreamWriter.writeEndElement(); } } From 0bd27847ab32a859fe0dcbb80553b40c26a247ff Mon Sep 17 00:00:00 2001 From: joewitt Date: Sat, 21 Mar 2015 10:18:52 -0400 Subject: [PATCH 063/116] NIFI-309 This closes #40 --- .../DynamicRelationship.java | 2 +- .../ReadsAttribute.java | 2 +- .../ReadsAttributes.java | 3 +- .../WritesAttribute.java | 2 +- .../WritesAttributes.java | 2 +- .../html/HtmlDocumentationWriter.java | 60 +++++++++---------- .../HtmlProcessorDocumentationWriter.java | 23 ++++--- .../example/FullyDocumentedProcessor.java | 8 +-- .../nifi/processors/hadoop/GetHDFS.java | 4 +- .../nifi/processors/hadoop/PutHDFS.java | 2 +- .../nifi/processors/kafka/GetKafka.java | 4 +- .../standard/Base64EncodeContent.java | 2 +- .../processors/standard/CompressContent.java | 4 +- .../processors/standard/DetectDuplicate.java | 2 +- .../processors/standard/DistributeLoad.java | 2 +- .../processors/standard/EvaluateJsonPath.java | 2 +- .../standard/EvaluateRegularExpression.java | 15 ++--- .../processors/standard/EvaluateXPath.java | 2 +- .../processors/standard/EvaluateXQuery.java | 2 +- .../standard/ExecuteStreamCommand.java | 4 +- .../nifi/processors/standard/GetFTP.java | 4 +- .../nifi/processors/standard/GetFile.java | 4 +- .../nifi/processors/standard/GetHTTP.java | 2 +- .../nifi/processors/standard/GetSFTP.java | 4 +- .../standard/HandleHttpRequest.java | 4 +- .../standard/HandleHttpResponse.java | 2 +- .../processors/standard/HashAttribute.java | 2 +- .../nifi/processors/standard/HashContent.java | 2 +- .../processors/standard/IdentifyMimeType.java | 2 +- .../nifi/processors/standard/InvokeHTTP.java | 4 +- .../processors/standard/MergeContent.java | 8 +-- .../processors/standard/MonitorActivity.java | 4 +- .../nifi/processors/standard/PostHTTP.java | 2 +- .../processors/standard/RouteOnAttribute.java | 2 +- .../processors/standard/RouteOnContent.java | 2 +- .../nifi/processors/standard/ScanContent.java | 2 +- .../processors/standard/SegmentContent.java | 4 +- .../processors/standard/SplitContent.java | 4 +- .../nifi/processors/standard/SplitText.java | 4 +- .../processors/standard/UnpackContent.java | 6 +- .../attributes/UpdateAttribute.java | 2 + 41 files changed, 106 insertions(+), 111 deletions(-) rename nifi/nifi-api/src/main/java/org/apache/nifi/annotation/{documentation => behavior}/DynamicRelationship.java (94%) rename nifi/nifi-api/src/main/java/org/apache/nifi/annotation/{documentation => behavior}/ReadsAttribute.java (94%) rename nifi/nifi-api/src/main/java/org/apache/nifi/annotation/{documentation => behavior}/ReadsAttributes.java (90%) rename nifi/nifi-api/src/main/java/org/apache/nifi/annotation/{documentation => behavior}/WritesAttribute.java (94%) rename nifi/nifi-api/src/main/java/org/apache/nifi/annotation/{documentation => behavior}/WritesAttributes.java (94%) diff --git a/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/documentation/DynamicRelationship.java b/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/behavior/DynamicRelationship.java similarity index 94% rename from nifi/nifi-api/src/main/java/org/apache/nifi/annotation/documentation/DynamicRelationship.java rename to nifi/nifi-api/src/main/java/org/apache/nifi/annotation/behavior/DynamicRelationship.java index e656bd3fc4..df0799fb6a 100644 --- a/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/documentation/DynamicRelationship.java +++ b/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/behavior/DynamicRelationship.java @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.nifi.annotation.documentation; +package org.apache.nifi.annotation.behavior; import java.lang.annotation.Documented; import java.lang.annotation.ElementType; diff --git a/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/documentation/ReadsAttribute.java b/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/behavior/ReadsAttribute.java similarity index 94% rename from nifi/nifi-api/src/main/java/org/apache/nifi/annotation/documentation/ReadsAttribute.java rename to nifi/nifi-api/src/main/java/org/apache/nifi/annotation/behavior/ReadsAttribute.java index 546db2d60e..d17ec2fed3 100644 --- a/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/documentation/ReadsAttribute.java +++ b/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/behavior/ReadsAttribute.java @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.nifi.annotation.documentation; +package org.apache.nifi.annotation.behavior; import java.lang.annotation.Documented; import java.lang.annotation.ElementType; diff --git a/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/documentation/ReadsAttributes.java b/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/behavior/ReadsAttributes.java similarity index 90% rename from nifi/nifi-api/src/main/java/org/apache/nifi/annotation/documentation/ReadsAttributes.java rename to nifi/nifi-api/src/main/java/org/apache/nifi/annotation/behavior/ReadsAttributes.java index 78ba4ecb94..dfa3513edf 100644 --- a/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/documentation/ReadsAttributes.java +++ b/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/behavior/ReadsAttributes.java @@ -14,8 +14,9 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.nifi.annotation.documentation; +package org.apache.nifi.annotation.behavior; +import org.apache.nifi.annotation.behavior.ReadsAttribute; import java.lang.annotation.Documented; import java.lang.annotation.ElementType; import java.lang.annotation.Inherited; diff --git a/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/documentation/WritesAttribute.java b/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/behavior/WritesAttribute.java similarity index 94% rename from nifi/nifi-api/src/main/java/org/apache/nifi/annotation/documentation/WritesAttribute.java rename to nifi/nifi-api/src/main/java/org/apache/nifi/annotation/behavior/WritesAttribute.java index 43f4947677..31d05d105b 100644 --- a/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/documentation/WritesAttribute.java +++ b/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/behavior/WritesAttribute.java @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.nifi.annotation.documentation; +package org.apache.nifi.annotation.behavior; import java.lang.annotation.Documented; import java.lang.annotation.ElementType; diff --git a/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/documentation/WritesAttributes.java b/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/behavior/WritesAttributes.java similarity index 94% rename from nifi/nifi-api/src/main/java/org/apache/nifi/annotation/documentation/WritesAttributes.java rename to nifi/nifi-api/src/main/java/org/apache/nifi/annotation/behavior/WritesAttributes.java index 140c85d0cb..4b03af9b53 100644 --- a/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/documentation/WritesAttributes.java +++ b/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/behavior/WritesAttributes.java @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.nifi.annotation.documentation; +package org.apache.nifi.annotation.behavior; import java.lang.annotation.Documented; import java.lang.annotation.ElementType; 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 c4261a1516..b1f426ae6a 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 @@ -72,14 +72,14 @@ public class HtmlDocumentationWriter implements DocumentationWriter { } /** - * Writes the head portion of the HTML documentation. + * 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 + * thrown if there was a problem writing to the stream */ protected void writeHead(final ConfigurableComponent configurableComponent, final XMLStreamWriter xmlStreamWriter) throws XMLStreamException { @@ -99,10 +99,10 @@ public class HtmlDocumentationWriter implements DocumentationWriter { } /** - * Gets the class name of the component. + * Gets the class name of the component. * * @param configurableComponent - * the component to describe + * the component to describe * @return the class name of the component */ protected String getTitle(final ConfigurableComponent configurableComponent) { @@ -111,7 +111,7 @@ public class HtmlDocumentationWriter implements DocumentationWriter { /** * Writes the body section of the documentation, this consists of the - * component description, the tags, and the PropertyDescriptors. + * component description, the tags, and the PropertyDescriptors. * * @param configurableComponent * the component to describe @@ -122,7 +122,7 @@ public class HtmlDocumentationWriter implements DocumentationWriter { * @throws XMLStreamException * thrown if there was a problem writing to the XML stream */ - private final void writeBody(final ConfigurableComponent configurableComponent, + private void writeBody(final ConfigurableComponent configurableComponent, final XMLStreamWriter xmlStreamWriter, final boolean hasAdditionalDetails) throws XMLStreamException { xmlStreamWriter.writeStartElement("body"); @@ -130,12 +130,12 @@ public class HtmlDocumentationWriter implements DocumentationWriter { writeTags(configurableComponent, xmlStreamWriter); writeProperties(configurableComponent, xmlStreamWriter); writeDynamicProperties(configurableComponent, xmlStreamWriter); - writeAdditionalBodyInfo(configurableComponent, xmlStreamWriter); - writeSeeAlso(configurableComponent, xmlStreamWriter); + writeAdditionalBodyInfo(configurableComponent, xmlStreamWriter); + writeSeeAlso(configurableComponent, xmlStreamWriter); xmlStreamWriter.writeEndElement(); } - /** + /** * Writes the list of components that may be linked from this component. * * @param configurableComponent @@ -194,12 +194,12 @@ public class HtmlDocumentationWriter implements DocumentationWriter { */ protected void writeAdditionalBodyInfo(final ConfigurableComponent configurableComponent, final XMLStreamWriter xmlStreamWriter) throws XMLStreamException { - + } /** - * Writes the tags attached to a ConfigurableComponent. - * + * Writes the tags attached to a ConfigurableComponent. + * * @param configurableComponent * @param xmlStreamWriter * @throws XMLStreamException @@ -222,7 +222,7 @@ public class HtmlDocumentationWriter implements DocumentationWriter { } /** - * Writes a description of the configurable component. + * Writes a description of the configurable component. * * @param configurableComponent * the component to describe @@ -232,7 +232,7 @@ public class HtmlDocumentationWriter implements DocumentationWriter { * whether there are additional details available as * 'additionalDetails.html' * @throws XMLStreamException - * thrown if there was a problem writing to the XML stream + * thrown if there was a problem writing to the XML stream */ protected void writeDescription(final ConfigurableComponent configurableComponent, final XMLStreamWriter xmlStreamWriter, final boolean hasAdditionalDetails) @@ -250,10 +250,10 @@ public class HtmlDocumentationWriter implements DocumentationWriter { /** * Gets a description of the ConfigurableComponent using the - * CapabilityDescription annotation. + * CapabilityDescription annotation. * * @param configurableComponent - * the component to describe + * the component to describe * @return a description of the configurableComponent */ protected String getDescription(final ConfigurableComponent configurableComponent) { @@ -271,14 +271,14 @@ public class HtmlDocumentationWriter implements DocumentationWriter { } /** - * Writes the PropertyDescriptors out as a table. + * 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 + * thrown if there was a problem writing to the XML Stream */ protected void writeProperties(final ConfigurableComponent configurableComponent, final XMLStreamWriter xmlStreamWriter) throws XMLStreamException { @@ -288,8 +288,8 @@ public class HtmlDocumentationWriter implements DocumentationWriter { 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"); + + "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 "); @@ -438,14 +438,14 @@ public class HtmlDocumentationWriter implements DocumentationWriter { /** * Interrogates a PropertyDescriptor to get a list of AllowableValues, if - * there are none, nothing is written to the stream. + * 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 + * thrown if there was a problem writing to the XML Stream */ protected void writeValidValues(XMLStreamWriter xmlStreamWriter, PropertyDescriptor property) throws XMLStreamException { @@ -462,10 +462,10 @@ public class HtmlDocumentationWriter implements DocumentationWriter { } xmlStreamWriter.writeEndElement(); - } else if (property.getControllerServiceDefinition() != null) { + } else if (property.getControllerServiceDefinition() != null) { Class controllerServiceClass = property .getControllerServiceDefinition(); - + writeSimpleElement(xmlStreamWriter, "strong", "Controller Service: "); xmlStreamWriter.writeEmptyElement("br"); xmlStreamWriter.writeCharacters(controllerServiceClass.getSimpleName()); @@ -474,7 +474,7 @@ public class HtmlDocumentationWriter implements DocumentationWriter { /** * Writes a begin element, then text, then end element for the element of a - * users choosing. Example: <p>text</p> + * users choosing. Example: <p>text</p> * * @param writer * the stream writer to use @@ -485,7 +485,7 @@ public class HtmlDocumentationWriter implements DocumentationWriter { * @param strong * whether the characters should be strong or not. * @throws XMLStreamException - * thrown if there was a problem writing to the stream. + * 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 { @@ -502,7 +502,7 @@ public class HtmlDocumentationWriter implements DocumentationWriter { /** * Writes a begin element, then text, then end element for the element of a - * users choosing. Example: <p>text</p> + * users choosing. Example: <p>text</p> * * @param writer * the stream writer to use @@ -511,7 +511,7 @@ public class HtmlDocumentationWriter implements DocumentationWriter { * @param characters * the characters to insert into the element * @throws XMLStreamException - * thrown if there was a problem writing to the stream + * 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 { @@ -519,7 +519,7 @@ public class HtmlDocumentationWriter implements DocumentationWriter { } /** - * A helper method to write a link + * A helper method to write a link * * @param xmlStreamWriter * the stream to write to @@ -528,7 +528,7 @@ public class HtmlDocumentationWriter implements DocumentationWriter { * @param location * the location of the link * @throws XMLStreamException - * thrown if there was a problem writing to the stream + * thrown if there was a problem writing to the stream */ protected void writeLink(final XMLStreamWriter xmlStreamWriter, final String text, final String location) throws XMLStreamException { 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 205e2d17a4..f210c7a610 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 @@ -17,26 +17,27 @@ package org.apache.nifi.documentation.html; import java.util.ArrayList; +import java.util.Arrays; import java.util.List; import javax.xml.stream.XMLStreamException; import javax.xml.stream.XMLStreamWriter; import org.apache.commons.lang3.StringUtils; -import org.apache.nifi.annotation.documentation.DynamicRelationship; -import org.apache.nifi.annotation.documentation.ReadsAttribute; -import org.apache.nifi.annotation.documentation.ReadsAttributes; -import org.apache.nifi.annotation.documentation.WritesAttribute; -import org.apache.nifi.annotation.documentation.WritesAttributes; +import org.apache.nifi.annotation.behavior.DynamicRelationship; +import org.apache.nifi.annotation.behavior.ReadsAttribute; +import org.apache.nifi.annotation.behavior.ReadsAttributes; +import org.apache.nifi.annotation.behavior.WritesAttribute; +import org.apache.nifi.annotation.behavior.WritesAttributes; 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. + * ConfigurableComponent as well as Relationship information. + * * - * */ public class HtmlProcessorDocumentationWriter extends HtmlDocumentationWriter { @@ -151,9 +152,7 @@ public class HtmlProcessorDocumentationWriter extends HtmlDocumentationWriter { ReadsAttributes readsAttributes = processor.getClass().getAnnotation(ReadsAttributes.class); if (readsAttributes != null) { - for (ReadsAttribute readAttribute : readsAttributes.value()) { - attributes.add(readAttribute); - } + attributes.addAll(Arrays.asList(readsAttributes.value())); } ReadsAttribute readsAttribute = processor.getClass().getAnnotation(ReadsAttribute.class); @@ -176,9 +175,7 @@ public class HtmlProcessorDocumentationWriter extends HtmlDocumentationWriter { WritesAttributes writesAttributes = processor.getClass().getAnnotation(WritesAttributes.class); if (writesAttributes != null) { - for (WritesAttribute writeAttribute : writesAttributes.value()) { - attributes.add(writeAttribute); - } + attributes.addAll(Arrays.asList(writesAttributes.value())); } WritesAttribute writeAttribute = processor.getClass().getAnnotation(WritesAttribute.class); 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 dd623adff1..cc59733188 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 @@ -24,12 +24,12 @@ import java.util.Set; import org.apache.nifi.annotation.behavior.DynamicProperty; import org.apache.nifi.annotation.documentation.CapabilityDescription; -import org.apache.nifi.annotation.documentation.DynamicRelationship; -import org.apache.nifi.annotation.documentation.ReadsAttribute; +import org.apache.nifi.annotation.behavior.DynamicRelationship; +import org.apache.nifi.annotation.behavior.ReadsAttribute; import org.apache.nifi.annotation.documentation.SeeAlso; import org.apache.nifi.annotation.documentation.Tags; -import org.apache.nifi.annotation.documentation.WritesAttribute; -import org.apache.nifi.annotation.documentation.WritesAttributes; +import org.apache.nifi.annotation.behavior.WritesAttribute; +import org.apache.nifi.annotation.behavior.WritesAttributes; import org.apache.nifi.components.AllowableValue; import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.processor.AbstractProcessor; diff --git a/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/GetHDFS.java b/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/GetHDFS.java index 12a6c9fe30..d763c29818 100644 --- a/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/GetHDFS.java +++ b/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/GetHDFS.java @@ -42,8 +42,8 @@ import org.apache.nifi.annotation.behavior.TriggerWhenEmpty; import org.apache.nifi.annotation.documentation.CapabilityDescription; import org.apache.nifi.annotation.documentation.SeeAlso; import org.apache.nifi.annotation.documentation.Tags; -import org.apache.nifi.annotation.documentation.WritesAttribute; -import org.apache.nifi.annotation.documentation.WritesAttributes; +import org.apache.nifi.annotation.behavior.WritesAttribute; +import org.apache.nifi.annotation.behavior.WritesAttributes; import org.apache.nifi.annotation.lifecycle.OnScheduled; import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.components.ValidationContext; diff --git a/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/PutHDFS.java b/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/PutHDFS.java index cb166c9a91..057f78671d 100644 --- a/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/PutHDFS.java +++ b/nifi/nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/PutHDFS.java @@ -35,7 +35,7 @@ import org.apache.hadoop.ipc.RemoteException; import org.apache.nifi.annotation.documentation.CapabilityDescription; import org.apache.nifi.annotation.documentation.SeeAlso; import org.apache.nifi.annotation.documentation.Tags; -import org.apache.nifi.annotation.documentation.WritesAttribute; +import org.apache.nifi.annotation.behavior.WritesAttribute; import org.apache.nifi.annotation.lifecycle.OnScheduled; import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.components.PropertyValue; diff --git a/nifi/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/java/org/apache/nifi/processors/kafka/GetKafka.java b/nifi/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/java/org/apache/nifi/processors/kafka/GetKafka.java index 4f894f93b7..63a816e75a 100644 --- a/nifi/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/java/org/apache/nifi/processors/kafka/GetKafka.java +++ b/nifi/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/java/org/apache/nifi/processors/kafka/GetKafka.java @@ -42,8 +42,8 @@ import kafka.message.MessageAndMetadata; import org.apache.nifi.annotation.behavior.SupportsBatching; import org.apache.nifi.annotation.documentation.CapabilityDescription; import org.apache.nifi.annotation.documentation.Tags; -import org.apache.nifi.annotation.documentation.WritesAttribute; -import org.apache.nifi.annotation.documentation.WritesAttributes; +import org.apache.nifi.annotation.behavior.WritesAttribute; +import org.apache.nifi.annotation.behavior.WritesAttributes; import org.apache.nifi.annotation.lifecycle.OnScheduled; import org.apache.nifi.annotation.lifecycle.OnStopped; import org.apache.nifi.annotation.lifecycle.OnUnscheduled; diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/Base64EncodeContent.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/Base64EncodeContent.java index cd272ff4e2..fa930d001b 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/Base64EncodeContent.java +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/Base64EncodeContent.java @@ -50,7 +50,7 @@ import org.apache.nifi.util.StopWatch; @SideEffectFree @SupportsBatching @Tags({"experimental", "encode", "base64"}) -@CapabilityDescription("Encodes the FlowFile content in base64") +@CapabilityDescription("Encodes or decodes content to and from base64") public class Base64EncodeContent extends AbstractProcessor { public static final String ENCODE_MODE = "Encode"; diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/CompressContent.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/CompressContent.java index 30cb091f76..5896a62e27 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/CompressContent.java +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/CompressContent.java @@ -40,9 +40,9 @@ import org.apache.nifi.annotation.behavior.EventDriven; import org.apache.nifi.annotation.behavior.SideEffectFree; import org.apache.nifi.annotation.behavior.SupportsBatching; import org.apache.nifi.annotation.documentation.CapabilityDescription; -import org.apache.nifi.annotation.documentation.ReadsAttribute; +import org.apache.nifi.annotation.behavior.ReadsAttribute; import org.apache.nifi.annotation.documentation.Tags; -import org.apache.nifi.annotation.documentation.WritesAttribute; +import org.apache.nifi.annotation.behavior.WritesAttribute; import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.flowfile.FlowFile; import org.apache.nifi.flowfile.attributes.CoreAttributes; diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/DetectDuplicate.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/DetectDuplicate.java index eb0d6c7f67..cd0a21bf67 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/DetectDuplicate.java +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/DetectDuplicate.java @@ -32,7 +32,7 @@ import org.apache.nifi.annotation.behavior.SupportsBatching; import org.apache.nifi.annotation.documentation.CapabilityDescription; import org.apache.nifi.annotation.documentation.SeeAlso; import org.apache.nifi.annotation.documentation.Tags; -import org.apache.nifi.annotation.documentation.WritesAttribute; +import org.apache.nifi.annotation.behavior.WritesAttribute; import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.distributed.cache.client.Deserializer; import org.apache.nifi.distributed.cache.client.DistributedMapCacheClient; 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 07b897083b..920041b919 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 @@ -36,7 +36,7 @@ import org.apache.nifi.annotation.behavior.SideEffectFree; import org.apache.nifi.annotation.behavior.SupportsBatching; import org.apache.nifi.annotation.behavior.TriggerWhenAnyDestinationAvailable; import org.apache.nifi.annotation.documentation.CapabilityDescription; -import org.apache.nifi.annotation.documentation.DynamicRelationship; +import org.apache.nifi.annotation.behavior.DynamicRelationship; import org.apache.nifi.annotation.documentation.Tags; import org.apache.nifi.annotation.lifecycle.OnScheduled; import org.apache.nifi.components.PropertyDescriptor; diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateJsonPath.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateJsonPath.java index d9b21423ad..e32eb126a0 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateJsonPath.java +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateJsonPath.java @@ -27,7 +27,7 @@ import org.apache.nifi.annotation.behavior.EventDriven; import org.apache.nifi.annotation.behavior.SideEffectFree; import org.apache.nifi.annotation.behavior.SupportsBatching; import org.apache.nifi.annotation.documentation.CapabilityDescription; -import org.apache.nifi.annotation.documentation.DynamicRelationship; +import org.apache.nifi.annotation.behavior.DynamicRelationship; import org.apache.nifi.annotation.documentation.Tags; import org.apache.nifi.annotation.lifecycle.OnRemoved; import org.apache.nifi.components.PropertyDescriptor; diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateRegularExpression.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateRegularExpression.java index 35436ca9c7..5d8af9f9a7 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateRegularExpression.java +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateRegularExpression.java @@ -48,23 +48,18 @@ import org.apache.nifi.annotation.documentation.Tags; import org.apache.nifi.processor.io.InputStreamCallback; import org.apache.nifi.processor.util.StandardValidators; import org.apache.commons.lang3.StringUtils; +import org.apache.nifi.annotation.documentation.SeeAlso; @EventDriven @SideEffectFree @SupportsBatching @Tags({"deprecated"}) @CapabilityDescription( - "WARNING: This has been deprecated and will be removed in 0.2.0. Use ExtractText instead.\n" - + "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.") + "WARNING: This has been deprecated and will be removed in 0.2.0. \n\n" + + "Use ExtractText instead.") +@SeeAlso(ExtractText.class) @Deprecated -@DynamicProperty(name="A FlowFile attribute", value="A regular expression with exactly one capturing group", description="Will update the specified FlowFile attribute with the group captured by the regular expression") +@DynamicProperty(name = "A FlowFile attribute", value = "A regular expression with exactly one capturing group", description = "Will update the specified FlowFile attribute with the group captured by the regular expression") public class EvaluateRegularExpression extends AbstractProcessor { public static final PropertyDescriptor CHARACTER_SET = new PropertyDescriptor.Builder() diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateXPath.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateXPath.java index 7852364202..55f55ff50b 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateXPath.java +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateXPath.java @@ -66,7 +66,7 @@ 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.WritesAttribute; +import org.apache.nifi.annotation.behavior.WritesAttribute; import org.apache.nifi.annotation.behavior.EventDriven; import org.apache.nifi.annotation.lifecycle.OnScheduled; import org.apache.nifi.annotation.behavior.DynamicProperty; diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateXQuery.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateXQuery.java index 8bc1f9feb5..ad1a2e42d2 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateXQuery.java +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateXQuery.java @@ -56,7 +56,7 @@ import org.apache.nifi.annotation.behavior.SideEffectFree; import org.apache.nifi.annotation.behavior.SupportsBatching; import org.apache.nifi.annotation.documentation.CapabilityDescription; import org.apache.nifi.annotation.documentation.Tags; -import org.apache.nifi.annotation.documentation.WritesAttribute; +import org.apache.nifi.annotation.behavior.WritesAttribute; import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.components.ValidationContext; import org.apache.nifi.components.ValidationResult; diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ExecuteStreamCommand.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ExecuteStreamCommand.java index c2f93650b1..4e6fd1ba85 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ExecuteStreamCommand.java +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ExecuteStreamCommand.java @@ -37,8 +37,8 @@ import org.apache.nifi.annotation.behavior.EventDriven; import org.apache.nifi.annotation.behavior.SupportsBatching; import org.apache.nifi.annotation.documentation.CapabilityDescription; import org.apache.nifi.annotation.documentation.Tags; -import org.apache.nifi.annotation.documentation.WritesAttribute; -import org.apache.nifi.annotation.documentation.WritesAttributes; +import org.apache.nifi.annotation.behavior.WritesAttribute; +import org.apache.nifi.annotation.behavior.WritesAttributes; import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.components.ValidationContext; import org.apache.nifi.components.ValidationResult; diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/GetFTP.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/GetFTP.java index 5077b7fe76..efce8fdeb1 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/GetFTP.java +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/GetFTP.java @@ -25,8 +25,8 @@ import org.apache.nifi.processor.ProcessContext; import org.apache.nifi.processor.ProcessorInitializationContext; import org.apache.nifi.annotation.documentation.CapabilityDescription; import org.apache.nifi.annotation.documentation.SeeAlso; -import org.apache.nifi.annotation.documentation.WritesAttribute; -import org.apache.nifi.annotation.documentation.WritesAttributes; +import org.apache.nifi.annotation.behavior.WritesAttribute; +import org.apache.nifi.annotation.behavior.WritesAttributes; import org.apache.nifi.annotation.behavior.SideEffectFree; import org.apache.nifi.annotation.documentation.Tags; import org.apache.nifi.processors.standard.util.FTPTransfer; diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/GetFile.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/GetFile.java index 881e25b2d6..f794095ace 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/GetFile.java +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/GetFile.java @@ -53,8 +53,8 @@ import org.apache.nifi.annotation.behavior.TriggerWhenEmpty; import org.apache.nifi.annotation.documentation.CapabilityDescription; import org.apache.nifi.annotation.documentation.SeeAlso; import org.apache.nifi.annotation.documentation.Tags; -import org.apache.nifi.annotation.documentation.WritesAttribute; -import org.apache.nifi.annotation.documentation.WritesAttributes; +import org.apache.nifi.annotation.behavior.WritesAttribute; +import org.apache.nifi.annotation.behavior.WritesAttributes; import org.apache.nifi.annotation.lifecycle.OnScheduled; import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.flowfile.FlowFile; diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/GetHTTP.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/GetHTTP.java index 1203cf9776..c1af0ea218 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/GetHTTP.java +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/GetHTTP.java @@ -69,7 +69,7 @@ import org.apache.http.impl.client.HttpClientBuilder; import org.apache.http.impl.conn.BasicHttpClientConnectionManager; import org.apache.nifi.annotation.documentation.CapabilityDescription; import org.apache.nifi.annotation.documentation.Tags; -import org.apache.nifi.annotation.documentation.WritesAttribute; +import org.apache.nifi.annotation.behavior.WritesAttribute; import org.apache.nifi.annotation.lifecycle.OnShutdown; import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.components.ValidationContext; diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/GetSFTP.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/GetSFTP.java index b3c09c5982..b9c348bafc 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/GetSFTP.java +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/GetSFTP.java @@ -25,8 +25,8 @@ import org.apache.nifi.annotation.behavior.SideEffectFree; import org.apache.nifi.annotation.documentation.CapabilityDescription; import org.apache.nifi.annotation.documentation.SeeAlso; import org.apache.nifi.annotation.documentation.Tags; -import org.apache.nifi.annotation.documentation.WritesAttribute; -import org.apache.nifi.annotation.documentation.WritesAttributes; +import org.apache.nifi.annotation.behavior.WritesAttribute; +import org.apache.nifi.annotation.behavior.WritesAttributes; import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.components.ValidationContext; import org.apache.nifi.components.ValidationResult; 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 37c504ad2f..76bfadf76d 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 @@ -47,8 +47,8 @@ import org.apache.commons.lang3.StringUtils; import org.apache.nifi.annotation.documentation.CapabilityDescription; import org.apache.nifi.annotation.documentation.SeeAlso; import org.apache.nifi.annotation.documentation.Tags; -import org.apache.nifi.annotation.documentation.WritesAttribute; -import org.apache.nifi.annotation.documentation.WritesAttributes; +import org.apache.nifi.annotation.behavior.WritesAttribute; +import org.apache.nifi.annotation.behavior.WritesAttributes; import org.apache.nifi.annotation.lifecycle.OnScheduled; import org.apache.nifi.annotation.lifecycle.OnStopped; import org.apache.nifi.components.AllowableValue; diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/HandleHttpResponse.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/HandleHttpResponse.java index cd0aebc5df..b50ce3bae6 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/HandleHttpResponse.java +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/HandleHttpResponse.java @@ -28,7 +28,7 @@ import javax.servlet.http.HttpServletResponse; import org.apache.nifi.annotation.behavior.DynamicProperty; import org.apache.nifi.annotation.documentation.CapabilityDescription; -import org.apache.nifi.annotation.documentation.ReadsAttribute; +import org.apache.nifi.annotation.behavior.ReadsAttribute; import org.apache.nifi.annotation.documentation.SeeAlso; import org.apache.nifi.annotation.documentation.Tags; import org.apache.nifi.components.PropertyDescriptor; diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/HashAttribute.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/HashAttribute.java index 816f98cd24..c5493e8842 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/HashAttribute.java +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/HashAttribute.java @@ -37,7 +37,7 @@ import org.apache.nifi.annotation.behavior.SideEffectFree; import org.apache.nifi.annotation.behavior.SupportsBatching; import org.apache.nifi.annotation.documentation.CapabilityDescription; import org.apache.nifi.annotation.documentation.Tags; -import org.apache.nifi.annotation.documentation.WritesAttribute; +import org.apache.nifi.annotation.behavior.WritesAttribute; import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.flowfile.FlowFile; import org.apache.nifi.logging.ProcessorLog; diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/HashContent.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/HashContent.java index 8daf615b78..99fd58b2b1 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/HashContent.java +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/HashContent.java @@ -32,7 +32,7 @@ import org.apache.nifi.annotation.behavior.EventDriven; import org.apache.nifi.annotation.behavior.SupportsBatching; import org.apache.nifi.annotation.documentation.CapabilityDescription; import org.apache.nifi.annotation.documentation.Tags; -import org.apache.nifi.annotation.documentation.WritesAttribute; +import org.apache.nifi.annotation.behavior.WritesAttribute; import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.flowfile.FlowFile; import org.apache.nifi.logging.ProcessorLog; diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/IdentifyMimeType.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/IdentifyMimeType.java index 4c560e854b..e31a4659b2 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/IdentifyMimeType.java +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/IdentifyMimeType.java @@ -28,7 +28,7 @@ import org.apache.nifi.annotation.behavior.SideEffectFree; import org.apache.nifi.annotation.behavior.SupportsBatching; import org.apache.nifi.annotation.documentation.CapabilityDescription; import org.apache.nifi.annotation.documentation.Tags; -import org.apache.nifi.annotation.documentation.WritesAttribute; +import org.apache.nifi.annotation.behavior.WritesAttribute; import org.apache.nifi.flowfile.FlowFile; import org.apache.nifi.flowfile.attributes.CoreAttributes; import org.apache.nifi.logging.ProcessorLog; diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/InvokeHTTP.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/InvokeHTTP.java index a2bab1d5d4..63ec0bbdbe 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/InvokeHTTP.java +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/InvokeHTTP.java @@ -53,8 +53,8 @@ import org.apache.nifi.annotation.behavior.DynamicProperty; import org.apache.nifi.annotation.behavior.SupportsBatching; import org.apache.nifi.annotation.documentation.CapabilityDescription; import org.apache.nifi.annotation.documentation.Tags; -import org.apache.nifi.annotation.documentation.WritesAttribute; -import org.apache.nifi.annotation.documentation.WritesAttributes; +import org.apache.nifi.annotation.behavior.WritesAttribute; +import org.apache.nifi.annotation.behavior.WritesAttributes; import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.flowfile.FlowFile; import org.apache.nifi.logging.ProcessorLog; diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/MergeContent.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/MergeContent.java index e19fc60614..9ca6470d5e 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/MergeContent.java +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/MergeContent.java @@ -40,12 +40,12 @@ import org.apache.commons.compress.archivers.tar.TarArchiveOutputStream; import org.apache.nifi.annotation.behavior.SideEffectFree; import org.apache.nifi.annotation.behavior.TriggerWhenEmpty; import org.apache.nifi.annotation.documentation.CapabilityDescription; -import org.apache.nifi.annotation.documentation.ReadsAttribute; -import org.apache.nifi.annotation.documentation.ReadsAttributes; +import org.apache.nifi.annotation.behavior.ReadsAttribute; +import org.apache.nifi.annotation.behavior.ReadsAttributes; import org.apache.nifi.annotation.documentation.SeeAlso; import org.apache.nifi.annotation.documentation.Tags; -import org.apache.nifi.annotation.documentation.WritesAttribute; -import org.apache.nifi.annotation.documentation.WritesAttributes; +import org.apache.nifi.annotation.behavior.WritesAttribute; +import org.apache.nifi.annotation.behavior.WritesAttributes; import org.apache.nifi.components.AllowableValue; import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.components.PropertyValue; diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/MonitorActivity.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/MonitorActivity.java index 1e74cb455d..ddb5330f60 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/MonitorActivity.java +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/MonitorActivity.java @@ -35,8 +35,8 @@ import org.apache.nifi.annotation.behavior.TriggerSerially; import org.apache.nifi.annotation.behavior.TriggerWhenEmpty; import org.apache.nifi.annotation.documentation.CapabilityDescription; import org.apache.nifi.annotation.documentation.Tags; -import org.apache.nifi.annotation.documentation.WritesAttribute; -import org.apache.nifi.annotation.documentation.WritesAttributes; +import org.apache.nifi.annotation.behavior.WritesAttribute; +import org.apache.nifi.annotation.behavior.WritesAttributes; import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.flowfile.FlowFile; import org.apache.nifi.flowfile.attributes.CoreAttributes; diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PostHTTP.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PostHTTP.java index b1017784e5..5f10772df9 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PostHTTP.java +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PostHTTP.java @@ -82,7 +82,7 @@ import org.apache.http.protocol.HttpCoreContext; import org.apache.http.util.EntityUtils; import org.apache.nifi.annotation.behavior.SupportsBatching; import org.apache.nifi.annotation.documentation.CapabilityDescription; -import org.apache.nifi.annotation.documentation.ReadsAttribute; +import org.apache.nifi.annotation.behavior.ReadsAttribute; import org.apache.nifi.annotation.documentation.Tags; import org.apache.nifi.annotation.lifecycle.OnScheduled; import org.apache.nifi.annotation.lifecycle.OnStopped; diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/RouteOnAttribute.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/RouteOnAttribute.java index 82798b8cba..374e6ec126 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/RouteOnAttribute.java +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/RouteOnAttribute.java @@ -31,7 +31,7 @@ import org.apache.nifi.annotation.behavior.EventDriven; import org.apache.nifi.annotation.behavior.SideEffectFree; import org.apache.nifi.annotation.behavior.SupportsBatching; import org.apache.nifi.annotation.documentation.CapabilityDescription; -import org.apache.nifi.annotation.documentation.DynamicRelationship; +import org.apache.nifi.annotation.behavior.DynamicRelationship; import org.apache.nifi.annotation.documentation.Tags; import org.apache.nifi.components.AllowableValue; import org.apache.nifi.components.PropertyDescriptor; diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/RouteOnContent.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/RouteOnContent.java index 34adfbb8e7..44d090be94 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/RouteOnContent.java +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/RouteOnContent.java @@ -41,7 +41,7 @@ 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.DynamicRelationship; +import org.apache.nifi.annotation.behavior.DynamicRelationship; import org.apache.nifi.annotation.behavior.DynamicProperty; import org.apache.nifi.annotation.behavior.EventDriven; import org.apache.nifi.annotation.behavior.SideEffectFree; diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ScanContent.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ScanContent.java index 55890e2b7b..ea2e6c291f 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ScanContent.java +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ScanContent.java @@ -39,7 +39,7 @@ import org.apache.nifi.annotation.behavior.SideEffectFree; import org.apache.nifi.annotation.behavior.SupportsBatching; import org.apache.nifi.annotation.documentation.CapabilityDescription; import org.apache.nifi.annotation.documentation.Tags; -import org.apache.nifi.annotation.documentation.WritesAttribute; +import org.apache.nifi.annotation.behavior.WritesAttribute; import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.flowfile.FlowFile; import org.apache.nifi.logging.ProcessorLog; diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/SegmentContent.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/SegmentContent.java index f2ce9a1e61..59aece0625 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/SegmentContent.java +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/SegmentContent.java @@ -31,8 +31,8 @@ import org.apache.nifi.annotation.behavior.SupportsBatching; import org.apache.nifi.annotation.documentation.CapabilityDescription; import org.apache.nifi.annotation.documentation.SeeAlso; import org.apache.nifi.annotation.documentation.Tags; -import org.apache.nifi.annotation.documentation.WritesAttribute; -import org.apache.nifi.annotation.documentation.WritesAttributes; +import org.apache.nifi.annotation.behavior.WritesAttribute; +import org.apache.nifi.annotation.behavior.WritesAttributes; import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.flowfile.FlowFile; import org.apache.nifi.flowfile.attributes.CoreAttributes; diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/SplitContent.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/SplitContent.java index 1c19805677..9838af7011 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/SplitContent.java +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/SplitContent.java @@ -35,8 +35,8 @@ import org.apache.nifi.annotation.behavior.SupportsBatching; import org.apache.nifi.annotation.documentation.CapabilityDescription; import org.apache.nifi.annotation.documentation.SeeAlso; import org.apache.nifi.annotation.documentation.Tags; -import org.apache.nifi.annotation.documentation.WritesAttribute; -import org.apache.nifi.annotation.documentation.WritesAttributes; +import org.apache.nifi.annotation.behavior.WritesAttribute; +import org.apache.nifi.annotation.behavior.WritesAttributes; import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.components.ValidationContext; import org.apache.nifi.components.ValidationResult; diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/SplitText.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/SplitText.java index da201887d2..d8f7400e13 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/SplitText.java +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/SplitText.java @@ -31,8 +31,8 @@ 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.SeeAlso; -import org.apache.nifi.annotation.documentation.WritesAttribute; -import org.apache.nifi.annotation.documentation.WritesAttributes; +import org.apache.nifi.annotation.behavior.WritesAttribute; +import org.apache.nifi.annotation.behavior.WritesAttributes; import org.apache.nifi.annotation.behavior.EventDriven; import org.apache.nifi.annotation.behavior.SideEffectFree; import org.apache.nifi.annotation.behavior.SupportsBatching; diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/UnpackContent.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/UnpackContent.java index 637b0c10a0..34b0693e98 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/UnpackContent.java +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/UnpackContent.java @@ -38,11 +38,11 @@ import org.apache.nifi.annotation.behavior.EventDriven; import org.apache.nifi.annotation.behavior.SideEffectFree; import org.apache.nifi.annotation.behavior.SupportsBatching; import org.apache.nifi.annotation.documentation.CapabilityDescription; -import org.apache.nifi.annotation.documentation.ReadsAttribute; +import org.apache.nifi.annotation.behavior.ReadsAttribute; import org.apache.nifi.annotation.documentation.SeeAlso; import org.apache.nifi.annotation.documentation.Tags; -import org.apache.nifi.annotation.documentation.WritesAttribute; -import org.apache.nifi.annotation.documentation.WritesAttributes; +import org.apache.nifi.annotation.behavior.WritesAttribute; +import org.apache.nifi.annotation.behavior.WritesAttributes; import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.flowfile.FlowFile; import org.apache.nifi.flowfile.attributes.CoreAttributes; diff --git a/nifi/nifi-nar-bundles/nifi-update-attribute-bundle/nifi-update-attribute-processor/src/main/java/org/apache/nifi/processors/attributes/UpdateAttribute.java b/nifi/nifi-nar-bundles/nifi-update-attribute-bundle/nifi-update-attribute-processor/src/main/java/org/apache/nifi/processors/attributes/UpdateAttribute.java index afdc4dbfbe..b7471dde78 100644 --- a/nifi/nifi-nar-bundles/nifi-update-attribute-bundle/nifi-update-attribute-processor/src/main/java/org/apache/nifi/processors/attributes/UpdateAttribute.java +++ b/nifi/nifi-nar-bundles/nifi-update-attribute-bundle/nifi-update-attribute-processor/src/main/java/org/apache/nifi/processors/attributes/UpdateAttribute.java @@ -60,6 +60,7 @@ import org.apache.nifi.update.attributes.Rule; import org.apache.nifi.update.attributes.FlowFilePolicy; import org.apache.nifi.update.attributes.serde.CriteriaSerDe; import org.apache.commons.lang3.StringUtils; +import org.apache.nifi.annotation.behavior.WritesAttribute; /** * This processor supports updating flowfile attributes and can do so @@ -115,6 +116,7 @@ import org.apache.commons.lang3.StringUtils; @CapabilityDescription("Updates the Attributes for a FlowFile by using the Attribute Expression Language") @DynamicProperty(name="A FlowFile attribute to update", value="The value to set it to", supportsExpressionLanguage=true, description="Updates a " + "FlowFile attribute specified by the Dynamic Property's key with the value specified by the Dynamic Property's value") +@WritesAttribute(attribute = "See additional details", description = "This processor may write zero or more attributes as described in additional details") public class UpdateAttribute extends AbstractProcessor implements Searchable { private final AtomicReference criteriaCache = new AtomicReference<>(null); From e05c9fd20e41cbb25f2e86b8a5b5791e6de65bcb Mon Sep 17 00:00:00 2001 From: Matt Gilman Date: Sun, 22 Mar 2015 23:13:07 -0400 Subject: [PATCH 064/116] Squashed commit of the following: commit 7e96bb903c4051613b5192e81aeaeef7997a9c1d Author: Matt Gilman Date: Sun Mar 22 23:09:18 2015 -0400 Conflicts: nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateRegularExpression.java nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ExtractText.java commit 7fc79a34b7bee4b92988a36c64f8585b7fec8d33 Author: Matt Gilman Date: Sun Mar 22 22:40:51 2015 -0400 NIFI-353: - Only showing up to 1.5kb of the content in the hex view. commit 36f11c3d9d8466fa3f207b5fc859375b33370b53 Author: Matt Gilman Date: Sun Mar 22 16:27:19 2015 -0400 NIFI-353: - Disabling the content viewer by default. commit f78f74dc8dcc21e8f01ae65cb17a6db80c2f3a6c Author: Matt Gilman Date: Sun Mar 22 16:24:45 2015 -0400 NIFI-353: - Adding error handling around the call into the content viewer extension. - Using forward instead of include when showing errors. commit b43fe6f935ce5697f551e5a3b30f7703b49f64a7 Author: Matt Gilman Date: Sun Mar 22 00:30:34 2015 -0400 NIFI-353: - Clean up. - Fixing authority check for DFM with check for Provenance. commit 61fc0467437a6fb0b9db4f1331699c23155aa3a6 Author: Matt Gilman Date: Sat Mar 21 22:19:19 2015 -0400 NIFI-353: - Fixing artifact versions. commit 2bd1a18f3ca5258f745ddf4681f79ea2741aaa78 Author: Matt Gilman Date: Fri Mar 20 20:59:44 2015 -0400 NIFI-353: - Fixing artifact versions. commit c2eaa192eae7e977ffd50033a007693f636a6322 Author: Matt Gilman Date: Fri Mar 20 20:15:40 2015 -0400 NIFI-353: - Javadocs. - Updating the styles of the content labels. commit 3410197b7d88444de5c6f74622d67a1b0cc39e6a Author: Matt Gilman Date: Fri Mar 20 19:40:02 2015 -0400 Squashed commit of the following: commit e88ed13d8d7221f5a91588f553d039d3917494be Merge: 93b361e 3f36236 Author: Mark Payne Date: Fri Mar 20 09:11:39 2015 -0400 Merge branch 'inputstream-callback-protection' of https://github.com/rowolabi/incubator-nifi into develop commit 93b361e69bd046d6b2e0f561d8b2231a4505b6a9 Merge: c9eb237 a6740a6 Author: Mark Payne Date: Thu Mar 19 11:49:11 2015 -0400 Merge branch 'develop' of http://git-wip-us.apache.org/repos/asf/incubator-nifi into develop commit c9eb237895a94a27fc6f760b82b16ac3e2cdab95 Author: Mark Payne Date: Thu Mar 19 11:40:01 2015 -0400 NIFI-443: Removed stopFunnel from ProcessGroup because we don't want to allow it commit 8b911c5aab2a4b8283510a3423e3c8962a533b96 Author: Mark Payne Date: Thu Mar 19 11:04:46 2015 -0400 NIFI-443: Always start funnels when added to process group, even when autoResumeState is false commit a6740a6e2c87f4c994d305db55c0777dc4f99976 Author: joewitt Date: Thu Mar 19 01:21:32 2015 -0400 NIFI-399 addressed items in the ticket commit ad18853b589d80331e2f4574bce35d79bce09c28 Author: joewitt Date: Wed Mar 18 10:59:13 2015 -0400 NIFI-399 initial port commit 3f3623647367421cd8eb318668144aca1afb9bf4 Author: Bobby Owolabi Date: Thu Mar 19 01:43:17 2015 -0400 NIFI-396 reverting accidentially modified whitespace in TestStandardProcessSession commit cd183be4410bdc88de7d3a0026452ab62eb10621 Author: Bobby Owolabi Date: Thu Mar 19 01:18:22 2015 -0400 NIFI-396 updated the javadocs of ProcessSession to reflect that a FlowFileAccessExcpetion will be thrown if an Input/Output Stream is attempted to be accessed after the callback is executed. commit e2760f8c980583d285137134e05c435c930fb4d2 Author: Bobby Owolabi Date: Thu Mar 19 00:54:24 2015 -0400 NIFI-396 added a DisableOnCloseInputStream class; modified StandardProcessSession to prevent access of the Input/OutputStreams after callbacks have been executed; updated tests commit 7272d0df58c23d099809bf96993b55d73c617476 Author: Bobby Owolabi Date: Wed Mar 18 23:30:57 2015 -0400 NIFI-396 created tests to demonstrate the situations where the ProcessSession throws an Exception and where it doesn't after it returns from the callback commit eb5ec703ba0d5c188822a37f6d7eed14af56a594 Author: Oscar de la Pena Date: Thu Mar 19 10:10:09 2015 +0800 Fixes incorrect messages count in Provenance reporter. Adds Unit test to verify fix commit b1873d86649d22fcf39956c93371be124a2e161b Author: Matt Gilman Date: Wed Mar 18 23:08:57 2015 -0400 NIFI-353: - Adjusting the layout of the file name and content type. commit 0ebb54a501825cb68134f009c4810fb79e49c39f Merge: ecbccae dea9e22 Author: Matt Gilman Date: Wed Mar 18 21:23:37 2015 -0400 Merge branch 'develop' into NIFI-353 commit ecbccae7343561e25acc9383b64b8c155f2c5700 Merge: 4c44843 1cca300 Author: Matt Gilman Date: Tue Mar 17 23:20:39 2015 -0400 Merge branch 'develop' into NIFI-353 commit 4c448436c5f84ac5bd4b3afad1b75f40c45c6d54 Author: Matt Gilman Date: Tue Mar 17 23:20:24 2015 -0400 NIFI-353: - Modifying the hexview plugin to address html escape issues and no longer rendering the base64 padding characters. commit 1a05c9db63cbbe21e08b40a87da14526653c41e1 Author: joewitt Date: Mon Mar 16 16:58:52 2015 -0400 NIFI-353 merged to latest dev post 002 release and fixed pom references. Viewer looks great commit 2b07b0bc1da9f4da6f40c31258c66afed30ffa4e Merge: f920902 eb757a4 Author: joewitt Date: Mon Mar 16 16:38:08 2015 -0400 Merge branch 'develop' into NIFI-353 commit f92090233fb3ad804cb1881d183592dfd30ffc99 Author: Matt Gilman Date: Sun Mar 15 23:51:31 2015 -0400 NIFI-353: - Addressing issues when running clustered. - Javadocs. commit 73a54eeb859fe9c8822141b59ee79eba8d1e6dff Author: Matt Gilman Date: Sun Mar 15 09:16:24 2015 -0400 NIFI-353: - Adding support for text/plain. commit c117a5c6f16c173ba971097fb9a14e9ed495f25b Author: Matt Gilman Date: Sun Mar 15 08:41:34 2015 -0400 NIFI-353: - Fixing dependency issue. - Setting the default content viewer path. - Restoring correct content type in the standard viewer META-INF. commit 12c867daea51d45b80767f82b1f8cf0ec249bb55 Merge: e7d77fe cc890e2 Author: Matt Gilman Date: Sat Mar 14 10:11:30 2015 -0400 Merge branch 'develop' into NIFI-353 commit e7d77fedbdac106803c553626050e380c8b51287 Author: Matt Gilman Date: Thu Mar 12 23:01:36 2015 -0400 NIFI-353 - Javadocs. - Cleaning up dependencies. commit a81e1ecbf85b34d5af2054b43118830f0d4fbfa3 Merge: 173177c 7198912 Author: Matt Gilman Date: Thu Mar 12 21:40:45 2015 -0400 Merge branch 'develop' into NIFI-353 commit 173177c918e6c07c1c98d807ecc7f8be6d0fa637 Author: Matt Gilman Date: Wed Mar 11 23:21:43 2015 -0400 NIFI-353: - Cleaning up error handling. - Showing the file name and content type. commit 098f9709dc0cac7a475d21c08d72de839260952c Author: Matt Gilman Date: Tue Mar 10 23:21:01 2015 -0400 NIFI-353: - Better error handling when unable to interpret the request, unable to find the content, and no viewer is registered for the detected content type. commit ee28e9de729dad477cc97b46d9a5e9cd9ef84609 Author: Matt Gilman Date: Sun Mar 8 22:57:15 2015 -0400 NIFI-353: - Replacing dependency on tika parsers with icu4j. commit a50a6b6e8bf393bb4d47672f69c13e1fa3bc202c Author: Matt Gilman Date: Sun Mar 8 22:21:49 2015 -0400 NIFI-353: - Allowing the user to toggle between viewing the original content, the formatted content, and the raw bytes in a hex dump. commit d100a2839bb2aef5af0de00f5e78ecc9c7f1ad0a Merge: 347e4e0 342ca17 Author: Matt Gilman Date: Sun Mar 8 13:54:24 2015 -0400 Merge branch 'develop' into NIFI-353 commit 347e4e024d4e990e6eb17a2101c31db79200f7a7 Author: Matt Gilman Date: Thu Mar 5 23:20:39 2015 -0500 NIFI-353: - Updating markup generation flow. - Adding a combo box to view the content in original, formatted, or hex form (still not functional). commit 37b5ca48f43cd3bb80b080e14c7cc2478da859aa Merge: 4819228 5e0026c Author: Matt Gilman Date: Wed Mar 4 21:44:10 2015 -0500 Merge branch 'develop' into NIFI-353 commit 48192289e492ed98bcc433925a70203570223c2a Merge: cc0b6fe 50744bf Author: Matt Gilman Date: Wed Feb 25 22:53:29 2015 -0500 Merge branch 'develop' into NIFI-353 commit cc0b6fe2f2b852457824357ff493b5c4e9d44ccd Author: Matt Gilman Date: Wed Feb 25 22:50:23 2015 -0500 NIFI-353: - Starting to moving the hex viewer into the main content viewer web application that comes bundled in framework. Previously it was only in the standard content viewer extension but we want to be able to render the content in hex for all types of data. commit 60c411de0dab6e0dc099e9b1fb04adfb1c2507f1 Author: Matt Gilman Date: Tue Feb 24 23:15:21 2015 -0500 NIFI-353: - Starting to add support for the hex viewer. commit efe8e06827488b1156edd5ea65e712d2eb675ef2 Author: Matt Gilman Date: Sun Feb 22 21:48:09 2015 -0500 NIFI-353: - Adding support for viewing xml documents. - Adding supporting to fold the json and xml documents. commit 1955926a857daede8091a761e984d22273ada235 Author: Matt Gilman Date: Sat Feb 21 21:49:48 2015 -0500 NIFI-353: - Allowing the content to be obtained by either stream or as a string (with the char encoding detected using tika). - Set the json viewer size during window resize events. commit 43f6e3c0585436176527344febc860586ded3b60 Author: Matt Gilman Date: Sat Feb 21 16:36:43 2015 -0500 NIFI-353: - Starting to add support for viewing JSON. - Updating codemirror to add support for JSON and XML. commit 8f54adf1c1e6cc581ee501d6f1182dac2ff63512 Merge: 605a05b 57b5d58 Author: Matt Gilman Date: Sat Feb 21 08:02:47 2015 -0500 Merge branch 'develop' into NIFI-353 commit 605a05b89f09bd16311d6321d6a1eeee881880b5 Author: Matt Gilman Date: Thu Feb 19 23:15:54 2015 -0500 NIFI-353: - Adding mime type detection using tika. commit bd9ef8431bc66f567d8e526e06f0635de2417254 Author: Matt Gilman Date: Wed Feb 18 23:35:28 2015 -0500 NIFI-353: - Continuing to wire up the content viewer controller to NiFi (standalone/clustered) and the content type specific renderer. commit 2334e4888e48764b93671b8ac2a00d26a07dc3bc Author: Matt Gilman Date: Tue Feb 17 23:00:07 2015 -0500 NIFI-353: - Updating the mark up on the content page. commit 78bab591d1c717077c7ee975c5ab4cf17c02d139 Merge: b1b2eaf 0047fa4 Author: Matt Gilman Date: Tue Feb 17 22:48:33 2015 -0500 Merge branch 'develop' into NIFI-353 commit b1b2eafe807c8f2e6cccb1a51736db8badd5d03b Author: Matt Gilman Date: Tue Feb 17 22:47:26 2015 -0500 NIFI-353: - Creating an interface for retrieving content. - Creating an interface for reading content bytes. - Integrating these concepts into the exiting content viewer controller. commit cd0a1bd42ac4b953cb1d8c979d7041373555882a Author: Matt Gilman Date: Mon Feb 16 17:08:32 2015 -0500 NIFI-353: - Renaming data-viewer to content-viewer. commit 48b1572f177e2292ddbacc92a3a5838cc45c7a42 Author: Matt Gilman Date: Mon Feb 16 15:45:15 2015 -0500 NIFI-353: - Renaming data-viewer to content-viewer. commit b50953d9d22881bcdc34edf1a34557d975b6dcc7 Author: Matt Gilman Date: Sun Feb 15 18:20:17 2015 -0500 NIFI-353: - Starting to integrate the data viewer controller into the Jetty Server. - Starting to set up the data viewer controller. - Starting to set up the standard data viewer. --- .../org/apache/nifi/web/ViewableContent.java | 74 +++++ nifi/nifi-assembly/pom.xml | 1 - .../nifi-web/nifi-jetty/pom.xml | 10 + .../apache/nifi/web/server/JettyServer.java | 106 +++++-- .../nifi-web/nifi-web-api/pom.xml | 5 + .../apache/nifi/web/NiFiServiceFacade.java | 1 - .../nifi/web/StandardNiFiContentAccess.java | 141 +++++++++ .../nifi/web/StandardNiFiServiceFacade.java | 3 - .../nifi/web/StandardNiFiWebContext.java | 2 +- .../nifi/web/api/ProvenanceResource.java | 2 +- .../nifi/web/controller/ControllerFacade.java | 2 +- .../main/resources/nifi-web-api-context.xml | 7 + .../nifi-web/nifi-web-content-access/pom.xml | 25 ++ .../org/apache/nifi/web/ContentAccess.java | 33 ++ .../nifi/web/ContentRequestContext.java | 51 ++++ .../apache/nifi/web}/DownloadableContent.java | 19 +- .../nifi-web-content-viewer/.gitignore | 1 + .../nifi-web/nifi-web-content-viewer/pom.xml | 91 ++++++ .../nifi/web/ContentViewerController.java | 284 ++++++++++++++++++ .../src/main/resources/META-INF/NOTICE | 19 ++ .../src/main/webapp/WEB-INF/jsp/footer.jsp | 20 ++ .../src/main/webapp/WEB-INF/jsp/header.jsp | 92 ++++++ .../src/main/webapp/WEB-INF/jsp/hexview.jsp | 32 ++ .../src/main/webapp/WEB-INF/jsp/no-viewer.jsp | 20 ++ .../src/main/webapp/WEB-INF/web.xml | 26 ++ .../src/main/webapp/css/main.css | 113 +++++++ .../src/main/webapp/js/hexview/LICENSE | 32 ++ .../webapp/js/hexview/hexview.default.css | 10 + .../src/main/webapp/js/hexview/hexview.js | 199 ++++++++++++ .../src/main/webapp/WEB-INF/web.xml | 11 + .../js/codemirror/addon/fold/foldgutter.css | 20 ++ .../codemirror/lib/codemirror-compressed.js | 14 +- .../webapp/js/jquery/combo/jquery.combo.css | 8 + .../nifi-framework/nifi-web/pom.xml | 8 + .../nifi-framework-bundle/pom.xml | 5 + .../nifi-standard-content-viewer/pom.xml | 71 +++++ .../web/StandardContentViewerController.java | 103 +++++++ .../src/main/resources/META-INF/NOTICE | 19 ++ .../main/webapp/META-INF/nifi-content-viewer | 3 + .../main/webapp/WEB-INF/jsp/codemirror.jsp | 47 +++ .../src/main/webapp/WEB-INF/web.xml | 29 ++ .../src/main/webapp/css/main.css | 20 ++ .../nifi-standard-nar/pom.xml | 5 + .../nifi-standard-bundle/pom.xml | 10 +- 44 files changed, 1754 insertions(+), 40 deletions(-) create mode 100644 nifi/nifi-api/src/main/java/org/apache/nifi/web/ViewableContent.java create mode 100644 nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiContentAccess.java create mode 100644 nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-content-access/pom.xml create mode 100644 nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-content-access/src/main/java/org/apache/nifi/web/ContentAccess.java create mode 100644 nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-content-access/src/main/java/org/apache/nifi/web/ContentRequestContext.java rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/{nifi-web-api/src/main/java/org/apache/nifi/web/util => nifi-web-content-access/src/main/java/org/apache/nifi/web}/DownloadableContent.java (82%) create mode 100755 nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-content-viewer/.gitignore create mode 100644 nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-content-viewer/pom.xml create mode 100644 nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-content-viewer/src/main/java/org/apache/nifi/web/ContentViewerController.java create mode 100644 nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-content-viewer/src/main/resources/META-INF/NOTICE create mode 100644 nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-content-viewer/src/main/webapp/WEB-INF/jsp/footer.jsp create mode 100644 nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-content-viewer/src/main/webapp/WEB-INF/jsp/header.jsp create mode 100644 nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-content-viewer/src/main/webapp/WEB-INF/jsp/hexview.jsp create mode 100644 nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-content-viewer/src/main/webapp/WEB-INF/jsp/no-viewer.jsp create mode 100644 nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-content-viewer/src/main/webapp/WEB-INF/web.xml create mode 100644 nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-content-viewer/src/main/webapp/css/main.css create mode 100644 nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-content-viewer/src/main/webapp/js/hexview/LICENSE create mode 100644 nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-content-viewer/src/main/webapp/js/hexview/hexview.default.css create mode 100644 nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-content-viewer/src/main/webapp/js/hexview/hexview.js create mode 100644 nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/codemirror/addon/fold/foldgutter.css create mode 100644 nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-content-viewer/pom.xml create mode 100644 nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-content-viewer/src/main/java/org/apache/nifi/web/StandardContentViewerController.java create mode 100644 nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-content-viewer/src/main/resources/META-INF/NOTICE create mode 100644 nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-content-viewer/src/main/webapp/META-INF/nifi-content-viewer create mode 100644 nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-content-viewer/src/main/webapp/WEB-INF/jsp/codemirror.jsp create mode 100644 nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-content-viewer/src/main/webapp/WEB-INF/web.xml create mode 100644 nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-content-viewer/src/main/webapp/css/main.css diff --git a/nifi/nifi-api/src/main/java/org/apache/nifi/web/ViewableContent.java b/nifi/nifi-api/src/main/java/org/apache/nifi/web/ViewableContent.java new file mode 100644 index 0000000000..f90221e97f --- /dev/null +++ b/nifi/nifi-api/src/main/java/org/apache/nifi/web/ViewableContent.java @@ -0,0 +1,74 @@ +/* + * 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.web; + +import java.io.IOException; +import java.io.InputStream; + +/** + * Interface for obtaining content from the NiFi content repository. + */ +public interface ViewableContent { + + public static final String CONTENT_REQUEST_ATTRIBUTE = "org.apache.nifi.web.content"; + + public enum DisplayMode { + Original, + Formatted, + Hex; + } + + /** + * The stream to the viewable content. The data stream can only be read once so + * an extension can call this method or getContent. + * + * @return + */ + InputStream getContentStream(); + + /** + * Gets the content as a string. The data stream can only be read once so + * an extension can call this method or getContentStream. + * + * @return + * @throws java.io.IOException + */ + String getContent() throws IOException; + + /** + * Returns the desired play mode. If the mode is Hex the + * framework will handle generating the mark up. The only + * values that an extension will see is Original or Formatted. + * + * @return + */ + DisplayMode getDisplayMode(); + + /** + * The contents file name. + * + * @return + */ + String getFileName(); + + /** + * The mime type of the content. + * + * @return + */ + String getContentType(); +} diff --git a/nifi/nifi-assembly/pom.xml b/nifi/nifi-assembly/pom.xml index ece7dbb21a..cae0f00a90 100644 --- a/nifi/nifi-assembly/pom.xml +++ b/nifi/nifi-assembly/pom.xml @@ -215,7 +215,6 @@ false - 30 sec 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 09a614dbcd..9f1cf99dee 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 @@ -113,6 +113,11 @@ nifi-client-dto compile + + org.apache.nifi + nifi-web-content-access + compile + org.apache.nifi nifi-custom-ui-utilities @@ -152,6 +157,11 @@ org.apache.nifi nifi-web-docs war + + + org.apache.nifi + nifi-web-content-viewer + war org.apache.nifi diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-jetty/src/main/java/org/apache/nifi/web/server/JettyServer.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-jetty/src/main/java/org/apache/nifi/web/server/JettyServer.java index 95052a7b10..54111a1df5 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-jetty/src/main/java/org/apache/nifi/web/server/JettyServer.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-jetty/src/main/java/org/apache/nifi/web/server/JettyServer.java @@ -52,6 +52,7 @@ import org.apache.nifi.util.NiFiProperties; import org.apache.nifi.web.NiFiWebContext; import org.apache.commons.collections4.CollectionUtils; import org.apache.commons.lang3.StringUtils; +import org.apache.nifi.web.ContentAccess; import org.eclipse.jetty.server.Connector; import org.eclipse.jetty.server.Handler; import org.eclipse.jetty.server.HttpConfiguration; @@ -99,7 +100,9 @@ public class JettyServer implements NiFiServer { private ExtensionMapping extensionMapping; private WebAppContext webApiContext; private WebAppContext webDocsContext; + private WebAppContext webContentViewerContext; private Collection customUiWebContexts; + private Collection contentViewerWebContexts; private final NiFiProperties props; /** @@ -164,6 +167,7 @@ public class JettyServer implements NiFiServer { File webApiWar = null; File webErrorWar = null; File webDocsWar = null; + File webContentViewerWar = null; List otherWars = new ArrayList<>(); for (File war : warToNarWorkingDirectoryLookup.keySet()) { if (war.getName().toLowerCase().startsWith("nifi-web-api")) { @@ -172,6 +176,8 @@ public class JettyServer implements NiFiServer { webErrorWar = war; } else if (war.getName().toLowerCase().startsWith("nifi-web-docs")) { webDocsWar = war; + } else if (war.getName().toLowerCase().startsWith("nifi-web-content-viewer")) { + webContentViewerWar = war; } else if (war.getName().toLowerCase().startsWith("nifi-web")) { webUiWar = war; } else { @@ -188,23 +194,29 @@ public class JettyServer implements NiFiServer { throw new RuntimeException("Unable to load nifi-web-docs WAR"); } else if (webErrorWar == null) { throw new RuntimeException("Unable to load nifi-web-error WAR"); + } else if (webContentViewerWar == null) { + throw new RuntimeException("Unable to load nifi-web-content-viewer WAR"); } // handlers for each war and init params for the web api final HandlerCollection handlers = new HandlerCollection(); - final Map initParams = new HashMap<>(); + final Map customUiMappings = new HashMap<>(); + final Map mimeTypeMappings = new HashMap<>(); final ClassLoader frameworkClassLoader = getClass().getClassLoader(); final ClassLoader jettyClassLoader = frameworkClassLoader.getParent(); // deploy the other wars if (CollectionUtils.isNotEmpty(otherWars)) { customUiWebContexts = new ArrayList<>(); + contentViewerWebContexts = new ArrayList<>(); + for (File war : otherWars) { // see if this war is a custom processor ui - List customUiProcessorTypes = getCustomUiProcessorTypes(war); + List customUiProcessorTypes = getWarExtensions(war, "META-INF/nifi-processor"); + List contentViewerMimeTypes = getWarExtensions(war, "META-INF/nifi-content-viewer"); - // only include wars that are for custom processor ui's - if (CollectionUtils.isNotEmpty(customUiProcessorTypes)) { + // only include wars that are for extensions + if (!customUiProcessorTypes.isEmpty() || !contentViewerMimeTypes.isEmpty()) { String warName = StringUtils.substringBeforeLast(war.getName(), "."); String warContextPath = String.format("/%s", warName); @@ -216,19 +228,27 @@ public class JettyServer implements NiFiServer { narClassLoaderForWar = jettyClassLoader; } - // create the custom ui web app context - WebAppContext customUiContext = loadWar(war, warContextPath, narClassLoaderForWar); + // create the extension web app context + WebAppContext extensionUiContext = loadWar(war, warContextPath, narClassLoaderForWar); - // hold on to a reference to all custom ui web app contexts - customUiWebContexts.add(customUiContext); + // also store it by type so we can populate the appropriate initialization parameters + if (!customUiProcessorTypes.isEmpty()) { + customUiWebContexts.add(extensionUiContext); + } else { + // record the mime type to web app mapping (need to handle type collision) + contentViewerWebContexts.add(extensionUiContext); + } // include custom ui web context in the handlers - handlers.addHandler(customUiContext); + handlers.addHandler(extensionUiContext); // add the initialization paramters for (String customUiProcessorType : customUiProcessorTypes) { // map the processor type to the custom ui path - initParams.put(customUiProcessorType, warContextPath); + customUiMappings.put(customUiProcessorType, warContextPath); + } + for (final String contentViewerMimeType : contentViewerMimeTypes) { + mimeTypeMappings.put(contentViewerMimeType, warContextPath); } } } @@ -239,10 +259,14 @@ public class JettyServer implements NiFiServer { // load the web api app webApiContext = loadWar(webApiWar, "/nifi-api", frameworkClassLoader); - Map webApiInitParams = webApiContext.getInitParams(); - webApiInitParams.putAll(initParams); + webApiContext.getInitParams().putAll(customUiMappings); handlers.addHandler(webApiContext); + // load the content viewer app + webContentViewerContext = loadWar(webContentViewerWar, "/nifi-content-viewer", frameworkClassLoader); + webContentViewerContext.getInitParams().putAll(mimeTypeMappings); + handlers.addHandler(webContentViewerContext); + // create a web app for the docs final String docsContextPath = "/nifi-docs"; @@ -292,18 +316,18 @@ public class JettyServer implements NiFiServer { } /** - * Loads the processor types that the specified war file is a custom UI for. + * Returns the extension in the specified WAR using the specified path. * - * @param warFile + * @param war * @return */ - private List getCustomUiProcessorTypes(final File warFile) { + private List getWarExtensions(final File war, final String path) { List processorTypes = new ArrayList<>(); JarFile jarFile = null; try { // load the jar file and attempt to find the nifi-processor entry - jarFile = new JarFile(warFile); - JarEntry jarEntry = jarFile.getJarEntry("META-INF/nifi-processor"); + jarFile = new JarFile(war); + JarEntry jarEntry = jarFile.getJarEntry(path); // ensure the nifi-processor entry was found if (jarEntry != null) { @@ -320,7 +344,7 @@ public class JettyServer implements NiFiServer { } } } catch (IOException ioe) { - logger.warn(String.format("Unable to inspect %s for a custom processor UI.", warFile)); + logger.warn(String.format("Unable to inspect %s for a custom processor UI.", war)); } finally { try { // close the jar file - which closes all input streams obtained via getInputStream above @@ -537,20 +561,48 @@ public class JettyServer implements NiFiServer { // ensure the appropriate wars deployed successfully before injecting the NiFi context and security filters - // this must be done after starting the server (and ensuring there were no start up failures) - if (webApiContext != null && CollectionUtils.isNotEmpty(customUiWebContexts)) { + if (webApiContext != null) { final ServletContext webApiServletContext = webApiContext.getServletHandler().getServletContext(); final WebApplicationContext webApplicationContext = WebApplicationContextUtils.getRequiredWebApplicationContext(webApiServletContext); - final NiFiWebContext NiFiWebContext = webApplicationContext.getBean("nifiWebContext", NiFiWebContext.class); - for (final WebAppContext customUiContext : customUiWebContexts) { - // set the NiFi context in each custom ui servlet context - final ServletContext customUiServletContext = customUiContext.getServletHandler().getServletContext(); - customUiServletContext.setAttribute("nifi-web-context", NiFiWebContext); + if (CollectionUtils.isNotEmpty(customUiWebContexts)) { + final NiFiWebContext niFiWebContext = webApplicationContext.getBean("nifiWebContext", NiFiWebContext.class); + + for (final WebAppContext customUiContext : customUiWebContexts) { + // set the NiFi context in each custom ui servlet context + final ServletContext customUiServletContext = customUiContext.getServletHandler().getServletContext(); + customUiServletContext.setAttribute("nifi-web-context", niFiWebContext); - // add the security filter to any custom ui wars + // add the security filter to any custom ui wars + final FilterHolder securityFilter = webApiContext.getServletHandler().getFilter("springSecurityFilterChain"); + if (securityFilter != null) { + customUiContext.addFilter(securityFilter, "/*", EnumSet.of(DispatcherType.REQUEST)); + } + } + } + + if (CollectionUtils.isNotEmpty(contentViewerWebContexts)) { + for (final WebAppContext contentViewerContext : contentViewerWebContexts) { + // add the security filter to any content viewer wars + final FilterHolder securityFilter = webApiContext.getServletHandler().getFilter("springSecurityFilterChain"); + if (securityFilter != null) { + contentViewerContext.addFilter(securityFilter, "/*", EnumSet.of(DispatcherType.REQUEST, DispatcherType.FORWARD, DispatcherType.INCLUDE)); + } + } + } + + // ensure the web content viewer war was loaded + if (webContentViewerContext != null) { + final ContentAccess contentAccess = webApplicationContext.getBean("contentAccess", ContentAccess.class); + + // add the content access + final ServletContext webContentViewerServletContext = webContentViewerContext.getServletHandler().getServletContext(); + webContentViewerServletContext.setAttribute("nifi-content-access", contentAccess); + + // add the security filter to the content viewer controller final FilterHolder securityFilter = webApiContext.getServletHandler().getFilter("springSecurityFilterChain"); if (securityFilter != null) { - customUiContext.addFilter(securityFilter, "/*", EnumSet.of(DispatcherType.REQUEST)); + webContentViewerContext.addFilter(securityFilter, "/*", EnumSet.of(DispatcherType.REQUEST, DispatcherType.FORWARD, DispatcherType.INCLUDE)); } } } @@ -560,7 +612,7 @@ public class JettyServer implements NiFiServer { final ServletContext webDocsServletContext = webDocsContext.getServletHandler().getServletContext(); webDocsServletContext.setAttribute("nifi-extension-mapping", extensionMapping); } - + // if this nifi is a node in a cluster, start the flow service and load the flow - the // flow service is loaded here for clustered nodes because the loading of the flow will // initialize the connection between the node and the NCM. if the node connects (starts 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 0bd34cf5e2..6a51838923 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 @@ -102,6 +102,11 @@ nifi-web-utils provided + + org.apache.nifi + nifi-web-content-access + provided + org.apache.nifi nifi-client-dto diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/NiFiServiceFacade.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/NiFiServiceFacade.java index ae6bf28f26..63d302c1cb 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/NiFiServiceFacade.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/NiFiServiceFacade.java @@ -65,7 +65,6 @@ import org.apache.nifi.web.api.dto.status.ControllerStatusDTO; import org.apache.nifi.web.api.dto.status.NodeStatusDTO; import org.apache.nifi.web.api.dto.status.ProcessGroupStatusDTO; import org.apache.nifi.web.api.dto.status.StatusHistoryDTO; -import org.apache.nifi.web.util.DownloadableContent; /** * Defines the NiFiServiceFacade interface. diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiContentAccess.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiContentAccess.java new file mode 100644 index 0000000000..70aad94417 --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiContentAccess.java @@ -0,0 +1,141 @@ +/* + * 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.web; + +import com.sun.jersey.api.client.ClientResponse; +import com.sun.jersey.core.util.MultivaluedMapImpl; +import java.io.Serializable; +import java.net.URI; +import java.net.URISyntaxException; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; +import javax.ws.rs.HttpMethod; +import javax.ws.rs.core.MultivaluedMap; +import org.apache.commons.lang3.StringUtils; +import org.apache.nifi.cluster.manager.NodeResponse; +import org.apache.nifi.cluster.manager.exception.UnknownNodeException; +import org.apache.nifi.cluster.manager.impl.WebClusterManager; +import org.apache.nifi.cluster.node.Node; +import org.apache.nifi.cluster.protocol.NodeIdentifier; +import org.apache.nifi.controller.repository.claim.ContentDirection; +import org.apache.nifi.util.NiFiProperties; +import org.apache.nifi.web.security.user.NiFiUserDetails; +import org.apache.nifi.web.util.WebUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.springframework.security.access.prepost.PreAuthorize; +import org.springframework.security.core.Authentication; +import org.springframework.security.core.context.SecurityContextHolder; + +/** + * + */ +public class StandardNiFiContentAccess implements ContentAccess { + + private static final Logger logger = LoggerFactory.getLogger(StandardNiFiContentAccess.class); + public static final String CLIENT_ID_PARAM = "clientId"; + + private NiFiProperties properties; + private NiFiServiceFacade serviceFacade; + private WebClusterManager clusterManager; + + @Override + @PreAuthorize("hasRole('ROLE_PROVENANCE')") + public DownloadableContent getContent(final ContentRequestContext request) { + // if clustered, send request to cluster manager + if (properties.isClusterManager()) { + // get the URI + URI dataUri; + try { + dataUri = new URI(request.getDataUri()); + } catch (final URISyntaxException use) { + throw new ClusterRequestException(use); + } + + // set the request parameters + final MultivaluedMap parameters = new MultivaluedMapImpl(); + parameters.add(CLIENT_ID_PARAM, request.getClientId()); + + // set the headers + final Map headers = new HashMap<>(); + if (StringUtils.isNotBlank(request.getProxiedEntitiesChain())) { + headers.put("X-ProxiedEntitiesChain", request.getProxiedEntitiesChain()); + } + + // add the user's authorities (if any) to the headers + final Authentication authentication = SecurityContextHolder.getContext().getAuthentication(); + if (authentication != null) { + final Object userDetailsObj = authentication.getPrincipal(); + if (userDetailsObj instanceof NiFiUserDetails) { + // serialize user details object + final String hexEncodedUserDetails = WebUtils.serializeObjectToHex((Serializable) userDetailsObj); + + // put serialized user details in header + headers.put("X-ProxiedEntityUserDetails", hexEncodedUserDetails); + } + } + + // get the target node and ensure it exists + final Node targetNode = clusterManager.getNode(request.getClusterNodeId()); + if (targetNode == null) { + throw new UnknownNodeException("The specified cluster node does not exist."); + } + + final Set targetNodes = new HashSet<>(); + targetNodes.add(targetNode.getNodeId()); + + // replicate the request to the specific node + final NodeResponse nodeResponse = clusterManager.applyRequest(HttpMethod.GET, dataUri, parameters, headers, targetNodes); + final ClientResponse clientResponse = nodeResponse.getClientResponse(); + final MultivaluedMap responseHeaders = clientResponse.getHeaders(); + + // get the file name + final String contentDisposition = responseHeaders.getFirst("Content-Disposition"); + final String filename = StringUtils.substringAfterLast(contentDisposition, "filename="); + + // get the content type + final String contentType = responseHeaders.getFirst("Content-Type"); + + // create the downloadable content + return new DownloadableContent(filename, contentType, clientResponse.getEntityInputStream()); + } else { + // example URI: http://localhost:8080/nifi-api/controller/provenance/events/1/content/input + final String eventDetails = StringUtils.substringAfterLast(request.getDataUri(), "events/"); + final String rawEventId = StringUtils.substringBefore(eventDetails, "/content/"); + final String rawDirection = StringUtils.substringAfterLast(eventDetails, "/content/"); + + // get the content type + final Long eventId = Long.parseLong(rawEventId); + final ContentDirection direction = ContentDirection.valueOf(rawDirection.toUpperCase()); + return serviceFacade.getContent(eventId, request.getDataUri(), direction); + } + } + + public void setProperties(NiFiProperties properties) { + this.properties = properties; + } + + public void setServiceFacade(NiFiServiceFacade serviceFacade) { + this.serviceFacade = serviceFacade; + } + + public void setClusterManager(WebClusterManager clusterManager) { + this.clusterManager = clusterManager; + } +} diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiServiceFacade.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiServiceFacade.java index c8683b0f0e..14640d8443 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiServiceFacade.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiServiceFacade.java @@ -16,8 +16,6 @@ */ package org.apache.nifi.web; -import org.apache.nifi.web.OptimisticLockingManager; -import org.apache.nifi.web.ConfigurationSnapshot; import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.Arrays; @@ -150,7 +148,6 @@ import org.apache.nifi.web.dao.ProcessorDAO; import org.apache.nifi.web.dao.RemoteProcessGroupDAO; import org.apache.nifi.web.dao.SnippetDAO; import org.apache.nifi.web.dao.TemplateDAO; -import org.apache.nifi.web.util.DownloadableContent; import org.apache.nifi.web.util.SnippetUtils; import org.apache.commons.collections4.CollectionUtils; diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiWebContext.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiWebContext.java index c1b1993979..d9fa9e3016 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiWebContext.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiWebContext.java @@ -89,7 +89,7 @@ public class StandardNiFiWebContext implements NiFiWebContext { } @Override - @PreAuthorize("hasAnyRole('ROLE_DFM')") + @PreAuthorize("hasRole('ROLE_DFM')") public void saveActions(final Collection processorActions) { Objects.requireNonNull(processorActions, "Actions cannot be null."); diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ProvenanceResource.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ProvenanceResource.java index f0b38a1b96..5fef27fb67 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ProvenanceResource.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ProvenanceResource.java @@ -74,7 +74,7 @@ import org.apache.nifi.web.api.request.ClientIdParameter; import org.apache.nifi.web.api.request.DateTimeParameter; import org.apache.nifi.web.api.request.IntegerParameter; import org.apache.nifi.web.api.request.LongParameter; -import org.apache.nifi.web.util.DownloadableContent; +import org.apache.nifi.web.DownloadableContent; import org.apache.commons.lang3.StringUtils; import org.codehaus.enunciate.jaxrs.TypeHint; diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerFacade.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerFacade.java index b009581079..117555a262 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerFacade.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerFacade.java @@ -108,7 +108,7 @@ import org.apache.nifi.web.api.dto.search.SearchResultsDTO; import org.apache.nifi.web.api.dto.status.ControllerStatusDTO; import org.apache.nifi.web.api.dto.status.ProcessGroupStatusDTO; import org.apache.nifi.web.api.dto.status.StatusHistoryDTO; -import org.apache.nifi.web.util.DownloadableContent; +import org.apache.nifi.web.DownloadableContent; import org.apache.commons.collections4.CollectionUtils; import org.apache.commons.lang3.StringUtils; import org.apache.nifi.admin.service.UserService; diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/resources/nifi-web-api-context.xml b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/resources/nifi-web-api-context.xml index 39677cacfd..a822442bca 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/resources/nifi-web-api-context.xml +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/resources/nifi-web-api-context.xml @@ -30,6 +30,13 @@ + + + + + + + diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-content-access/pom.xml b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-content-access/pom.xml new file mode 100644 index 0000000000..c5c12d3ba3 --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-content-access/pom.xml @@ -0,0 +1,25 @@ + + + + 4.0.0 + + org.apache.nifi + nifi-web + 0.1.0-incubating-SNAPSHOT + + org.apache.nifi + nifi-web-content-access + diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-content-access/src/main/java/org/apache/nifi/web/ContentAccess.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-content-access/src/main/java/org/apache/nifi/web/ContentAccess.java new file mode 100644 index 0000000000..a093c59163 --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-content-access/src/main/java/org/apache/nifi/web/ContentAccess.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.web; + +/** + * Provides access to content within NiFi. + * + * @author unattributed + */ +public interface ContentAccess { + + /** + * Gets the content for the specified claim. + * + * @param request + * @return + */ + DownloadableContent getContent(ContentRequestContext request); +} diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-content-access/src/main/java/org/apache/nifi/web/ContentRequestContext.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-content-access/src/main/java/org/apache/nifi/web/ContentRequestContext.java new file mode 100644 index 0000000000..f5744eea90 --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-content-access/src/main/java/org/apache/nifi/web/ContentRequestContext.java @@ -0,0 +1,51 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.web; + +/** + * A request for content. + */ +public interface ContentRequestContext { + + /** + * The URI to the data. + * + * @return + */ + String getDataUri(); + + /** + * If clustered, this is the id of the node the data resides on. + * + * @return + */ + String getClusterNodeId(); + + /** + * The client id for the user making the request. + * + * @return + */ + String getClientId(); + + /** + * The proxy chain for the current request, if applicable. + * + * @return + */ + String getProxiedEntitiesChain(); +} diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/util/DownloadableContent.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-content-access/src/main/java/org/apache/nifi/web/DownloadableContent.java similarity index 82% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/util/DownloadableContent.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-content-access/src/main/java/org/apache/nifi/web/DownloadableContent.java index 9bb54f1f69..a23673f5e8 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/util/DownloadableContent.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-content-access/src/main/java/org/apache/nifi/web/DownloadableContent.java @@ -14,12 +14,12 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.nifi.web.util; +package org.apache.nifi.web; import java.io.InputStream; /** - * + * Represents content that can be downloaded. */ public final class DownloadableContent { @@ -33,14 +33,29 @@ public final class DownloadableContent { this.content = content; } + /** + * The filename of the content. + * + * @return + */ public String getFilename() { return filename; } + /** + * The content type of the content. + * + * @return + */ public String getType() { return type; } + /** + * The content stream. + * + * @return + */ public InputStream getContent() { return content; } diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-content-viewer/.gitignore b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-content-viewer/.gitignore new file mode 100755 index 0000000000..ea8c4bf7f3 --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-content-viewer/.gitignore @@ -0,0 +1 @@ +/target diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-content-viewer/pom.xml b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-content-viewer/pom.xml new file mode 100644 index 0000000000..75464c25f9 --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-content-viewer/pom.xml @@ -0,0 +1,91 @@ + + + 4.0.0 + + org.apache.nifi + nifi-web + 0.1.0-incubating-SNAPSHOT + + org.apache.nifi + nifi-web-content-viewer + war + + + org.apache.nifi + nifi-api + + + org.apache.nifi + nifi-web-content-access + provided + + + org.apache.nifi + nifi-utils + provided + + + org.apache.commons + commons-lang3 + provided + + + commons-codec + commons-codec + provided + + + commons-io + commons-io + provided + + + org.springframework.security + spring-security-core + provided + + + com.ibm.icu + icu4j + 54.1.1 + + + org.apache.tika + tika-core + 1.7 + + + javax.servlet.jsp + javax.servlet.jsp-api + provided + + + javax.el + javax.el-api + provided + + + javax.servlet.jsp.jstl + javax.servlet.jsp.jstl-api + provided + + + javax.servlet + javax.servlet-api + provided + + + diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-content-viewer/src/main/java/org/apache/nifi/web/ContentViewerController.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-content-viewer/src/main/java/org/apache/nifi/web/ContentViewerController.java new file mode 100644 index 0000000000..d9b082d9ba --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-content-viewer/src/main/java/org/apache/nifi/web/ContentViewerController.java @@ -0,0 +1,284 @@ +/* + * 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.web; + +import com.ibm.icu.text.CharsetDetector; +import com.ibm.icu.text.CharsetMatch; +import java.io.BufferedInputStream; +import java.io.IOException; +import java.io.InputStream; + +import javax.servlet.ServletContext; +import javax.servlet.ServletException; +import javax.servlet.http.HttpServlet; +import javax.servlet.http.HttpServletRequest; +import javax.servlet.http.HttpServletResponse; +import org.apache.commons.codec.binary.Base64; +import org.apache.commons.io.IOUtils; +import org.apache.commons.lang3.StringUtils; +import org.apache.nifi.stream.io.StreamUtils; +import org.apache.nifi.web.ViewableContent.DisplayMode; +import org.apache.tika.detect.DefaultDetector; +import org.apache.tika.io.TikaInputStream; +import org.apache.tika.metadata.Metadata; +import org.apache.tika.mime.MediaType; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.springframework.security.access.AccessDeniedException; + +/** + * Controller servlet for viewing content. This is responsible for generating + * the markup for the header and footer of the page. Included in that is the + * combo that allows the user to choose how they wait to view the data + * (original, formatted, hex). If a data viewer is registered for the detected + * content type, it will include the markup it generates in the response. + */ +public class ContentViewerController extends HttpServlet { + + private static final Logger logger = LoggerFactory.getLogger(ContentViewerController.class); + + // 1.5kb - multiple of 12 (3 bytes = 4 base 64 encoded chars) + private final static int BUFFER_LENGTH = 1536; + + /** + * Gets the content and defers to registered viewers to generate the markup. + * + * @param request servlet request + * @param response servlet response + * @throws ServletException if a servlet-specific error occurs + * @throws IOException if an I/O error occurs + */ + @Override + protected void doGet(final HttpServletRequest request, final HttpServletResponse response) throws ServletException, IOException { + // get the content + final ServletContext servletContext = request.getServletContext(); + final ContentAccess contentAccess = (ContentAccess) servletContext.getAttribute("nifi-content-access"); + + // get the content + final DownloadableContent downloadableContent; + try { + downloadableContent = contentAccess.getContent(getContentRequest(request)); + } catch (final ResourceNotFoundException rnfe) { + request.setAttribute("title", "Error"); + request.setAttribute("messages", "Unable to find the specified content"); + + // forward to the error page + final ServletContext viewerContext = servletContext.getContext("/nifi"); + viewerContext.getRequestDispatcher("/message").forward(request, response); + return; + } catch (final AccessDeniedException ade) { + request.setAttribute("title", "Acess Denied"); + request.setAttribute("messages", "Unable to approve access to the specified content: " + ade.getMessage()); + + // forward to the error page + final ServletContext viewerContext = servletContext.getContext("/nifi"); + viewerContext.getRequestDispatcher("/message").forward(request, response); + return; + } catch (final Exception e) { + request.setAttribute("title", "Error"); + request.setAttribute("messages", "An unexcepted error has occurred: " + e.getMessage()); + + // forward to the error page + final ServletContext viewerContext = servletContext.getContext("/nifi"); + viewerContext.getRequestDispatcher("/message").forward(request, response); + return; + } + + // determine how we want to view the data + String mode = request.getParameter("mode"); + + // if the name isn't set, use original + if (mode == null) { + mode = DisplayMode.Original.name(); + } + + // determine the display mode + final DisplayMode displayMode; + try { + displayMode = DisplayMode.valueOf(mode); + } catch (final IllegalArgumentException iae) { + request.setAttribute("title", "Error"); + request.setAttribute("messages", "Invalid display mode: " + mode); + + // forward to the error page + final ServletContext viewerContext = servletContext.getContext("/nifi"); + viewerContext.getRequestDispatcher("/message").forward(request, response); + return; + } + + // buffer the content to support reseting in case we need to detect the content type or char encoding + final BufferedInputStream bis = new BufferedInputStream(downloadableContent.getContent()); + + // detect the content type + final DefaultDetector detector = new DefaultDetector(); + + // create the stream for tika to process, buffered to support reseting + final TikaInputStream tikaStream = TikaInputStream.get(bis); + + // provide a hint based on the filename + final Metadata metadata = new Metadata(); + metadata.set(Metadata.RESOURCE_NAME_KEY, downloadableContent.getFilename()); + + // Get mime type + final MediaType mediatype = detector.detect(tikaStream, metadata); + final String mimeType = mediatype.toString(); + + // add attributes needed for the header + final StringBuffer requestUrl = request.getRequestURL(); + request.setAttribute("requestUrl", requestUrl.toString()); + request.setAttribute("dataRef", request.getParameter("ref")); + request.setAttribute("filename", downloadableContent.getFilename()); + request.setAttribute("contentType", mimeType); + + // generate the header + request.getRequestDispatcher("/WEB-INF/jsp/header.jsp").include(request, response); + + // remove the attributes needed for the header + request.removeAttribute("requestUrl"); + request.removeAttribute("dataRef"); + request.removeAttribute("filename"); + request.removeAttribute("contentType"); + + // generate the markup for the content based on the display mode + if (DisplayMode.Hex.equals(displayMode)) { + final byte[] buffer = new byte[BUFFER_LENGTH]; + final int read = StreamUtils.fillBuffer(bis, buffer, false); + + // trim the byte array if necessary + byte[] bytes = buffer; + if (read != buffer.length) { + bytes = new byte[read]; + System.arraycopy(buffer, 0, bytes, 0, read); + } + + // convert bytes into the base 64 bytes + final String base64 = Base64.encodeBase64String(bytes); + + // defer to the jsp + request.setAttribute("content", base64); + request.getRequestDispatcher("/WEB-INF/jsp/hexview.jsp").include(request, response); + } else { + // lookup a viewer for the content + final String contentViewerUri = servletContext.getInitParameter(mimeType); + + // handle no viewer for content type + if (contentViewerUri == null) { + request.getRequestDispatcher("/WEB-INF/jsp/no-viewer.jsp").include(request, response); + } else { + // create a request attribute for accessing the content + request.setAttribute(ViewableContent.CONTENT_REQUEST_ATTRIBUTE, new ViewableContent() { + @Override + public InputStream getContentStream() { + return bis; + } + + @Override + public String getContent() throws IOException { + // detect the charset + final CharsetDetector detector = new CharsetDetector(); + detector.setText(bis); + detector.enableInputFilter(true); + final CharsetMatch match = detector.detect(); + + // ensure we were able to detect the charset + if (match == null) { + throw new IOException("Unable to detect character encoding."); + } + + // convert the stream using the detected charset + return IOUtils.toString(bis, match.getName()); + } + + @Override + public ViewableContent.DisplayMode getDisplayMode() { + return displayMode; + } + + @Override + public String getFileName() { + return downloadableContent.getFilename(); + } + + @Override + public String getContentType() { + return mimeType; + } + }); + + try { + // generate the content + final ServletContext viewerContext = servletContext.getContext(contentViewerUri); + viewerContext.getRequestDispatcher("/view-content").include(request, response); + } catch (final Exception e) { + String message = e.getMessage() != null ? e.getMessage() : e.toString(); + message = "Unable to generate view of data: " + message; + + // log the error + logger.error(message); + if (logger.isDebugEnabled()) { + logger.error(StringUtils.EMPTY, e); + } + + // populate the request attributes + request.setAttribute("title", "Error"); + request.setAttribute("messages", message); + + // forward to the error page + final ServletContext viewerContext = servletContext.getContext("/nifi"); + viewerContext.getRequestDispatcher("/message").forward(request, response); + return; + } + + // remove the request attribute + request.removeAttribute(ViewableContent.CONTENT_REQUEST_ATTRIBUTE); + } + } + + // generate footer + request.getRequestDispatcher("/WEB-INF/jsp/footer.jsp").include(request, response); + } + + /** + * Get the content request context based on the specified request. + * @param request + * @return + */ + private ContentRequestContext getContentRequest(final HttpServletRequest request) { + return new ContentRequestContext() { + @Override + public String getDataUri() { + return request.getParameter("ref"); + } + + @Override + public String getClusterNodeId() { + final String ref = request.getParameter("ref"); + return StringUtils.substringAfterLast(ref, "clusterNodeId="); + } + + @Override + public String getClientId() { + return request.getParameter("clientId"); + } + + @Override + public String getProxiedEntitiesChain() { + return request.getHeader("X-ProxiedEntitiesChain"); + } + }; + } +} diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-content-viewer/src/main/resources/META-INF/NOTICE b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-content-viewer/src/main/resources/META-INF/NOTICE new file mode 100644 index 0000000000..d91a95247a --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-content-viewer/src/main/resources/META-INF/NOTICE @@ -0,0 +1,19 @@ +nifi-web-docs +Copyright 2014-2015 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + +=========================================== +Apache Software License v2 +=========================================== + +The following binary components are provided under the Apache Software License v2 + + (ASLv2) Apache Commons Lang + The following NOTICE information applies: + Apache Commons Lang + Copyright 2001-2014 The Apache Software Foundation + + This product includes software from the Spring Framework, + under the Apache License 2.0 (see: StringUtils.containsWhitespace()) diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-content-viewer/src/main/webapp/WEB-INF/jsp/footer.jsp b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-content-viewer/src/main/webapp/WEB-INF/jsp/footer.jsp new file mode 100644 index 0000000000..5288315ce1 --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-content-viewer/src/main/webapp/WEB-INF/jsp/footer.jsp @@ -0,0 +1,20 @@ +<%-- + 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. +--%> +<%@ page contentType="text/html" pageEncoding="UTF-8" session="false" %> + + + \ No newline at end of file diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-content-viewer/src/main/webapp/WEB-INF/jsp/header.jsp b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-content-viewer/src/main/webapp/WEB-INF/jsp/header.jsp new file mode 100644 index 0000000000..82382f619d --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-content-viewer/src/main/webapp/WEB-INF/jsp/header.jsp @@ -0,0 +1,92 @@ +<%-- + 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. +--%> +<%@ page contentType="text/html" pageEncoding="UTF-8" session="false" %> + + + + + + NiFi + + + + + + + + + + +
      View as
      +
      +
      ${filename}
      +
      ${contentType}
      +
      \ No newline at end of file diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-content-viewer/src/main/webapp/WEB-INF/jsp/hexview.jsp b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-content-viewer/src/main/webapp/WEB-INF/jsp/hexview.jsp new file mode 100644 index 0000000000..c6e7f382ff --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-content-viewer/src/main/webapp/WEB-INF/jsp/hexview.jsp @@ -0,0 +1,32 @@ +<%-- + 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. +--%> +<%@ page contentType="text/html" pageEncoding="UTF-8" session="false" %> + + + + +
      + ${content} +
      + + + + + +
      +
      +
      Showing up to 1.5kb
      \ No newline at end of file diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-content-viewer/src/main/webapp/WEB-INF/jsp/no-viewer.jsp b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-content-viewer/src/main/webapp/WEB-INF/jsp/no-viewer.jsp new file mode 100644 index 0000000000..8eddbe2546 --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-content-viewer/src/main/webapp/WEB-INF/jsp/no-viewer.jsp @@ -0,0 +1,20 @@ +<%-- + 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. +--%> +<%@ page contentType="text/html" pageEncoding="UTF-8" session="false" %> +
      + No viewer is registered for this content type. +
      \ No newline at end of file diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-content-viewer/src/main/webapp/WEB-INF/web.xml b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-content-viewer/src/main/webapp/WEB-INF/web.xml new file mode 100644 index 0000000000..77f08537db --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-content-viewer/src/main/webapp/WEB-INF/web.xml @@ -0,0 +1,26 @@ + + + + nifi-content-viewer + + ContentViewerController + org.apache.nifi.web.ContentViewerController + + + ContentViewerController + + + diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-content-viewer/src/main/webapp/css/main.css b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-content-viewer/src/main/webapp/css/main.css new file mode 100644 index 0000000000..a66198de20 --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-content-viewer/src/main/webapp/css/main.css @@ -0,0 +1,113 @@ +/* + * 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. + */ +* { + margin: 0; + padding: 0; +} + +#view-as-label { + position: absolute; + top: 72px; + left: 144px; + line-height: 24px; + font-size: 12px; +} + +#view-as { + position: absolute; + top: 72px; + left: 200px; + width: 80px; + height: 18px; +} + +#content-filename { + position: absolute; + top: 58px; + right: 50px; + line-height: 24px; + font-size: 12px; +} + +#content-type { + position: absolute; + top: 75px; + right: 50px; + line-height: 24px; + font-size: 12px; +} + +.content-label { + font-weight: bold; + margin-right: 8px; +} + +.pointer { + cursor: pointer; +} + +.hidden { + display: none; +} + +/* hex viewer */ + +#hexview-content { + position: absolute; + right: 50px; + bottom: 50px; + left: 100px; + top: 100px; + border: 1px solid #aaa; + overflow: auto; + background-color: #fff; +} + +#hexview-content table.hexviewerwindow_table { + border: none; + margin-left: 0; + background-color: #fff; +} + +#hexview-content td { + padding: 2px; +} + +#trancation-message { + position: absolute; + left: 100px; + bottom: 35px; + color: #666; + font-style: italic; + font-size: 11px; +} + +/* no viewer */ + +#no-viewer { + position: absolute; + right: 50px; + bottom: 50px; + left: 100px; + top: 100px; + border: 1px solid #aaa; + overflow: auto; + background-color: #fff; + font-style: italic; + padding: 5px; + font-size: 13px; +} \ No newline at end of file diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-content-viewer/src/main/webapp/js/hexview/LICENSE b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-content-viewer/src/main/webapp/js/hexview/LICENSE new file mode 100644 index 0000000000..95e1332ccf --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-content-viewer/src/main/webapp/js/hexview/LICENSE @@ -0,0 +1,32 @@ + +HexViewJS License +----------------- + +HexViewJS is written by Nick McVeity and is +licensed under the terms of the MIT license reproduced below. + +======================================================================== + +Copyright (c) 2010 Nick McVeity + +Permission is hereby granted, free of charge, to any person +obtaining a copy of this software and associated documentation +files (the "Software"), to deal in the Software without restriction, +including without limitation the rights to use, copy, modify, merge, +publish, distribute, sublicense, and/or sell copies of the Software, +and to permit persons to whom the Software is furnished to do so, +subject to the following conditions: + +The above copyright notice and this permission notice shall be +included in all copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, +EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF +MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. +IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY +CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, +TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE +SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. + +======================================================================== + diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-content-viewer/src/main/webapp/js/hexview/hexview.default.css b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-content-viewer/src/main/webapp/js/hexview/hexview.default.css new file mode 100644 index 0000000000..581161ee98 --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-content-viewer/src/main/webapp/js/hexview/hexview.default.css @@ -0,0 +1,10 @@ +.hexviewerwindow { font-family: monospace; background-color: #F2F2F2;} +div.hexviewerwindow { padding: 20px; } +.hexviewerwindow_table { border-collapse:collapse; border: 5px solid grey; margin-left: 16px; caption-side:bottom; } +.hexviewerwindow_offset {background: #A9D0F5; padding-right: 8px; } +.hexviewerwindow_visual {background: #A9F5F2; padding-left: 8px; } +.hexviewerwindow_code {} +.hexviewerwindow_code_hi {background: #F4FA58; } +.hexviewerwindow_border_start {border-left: solid #E0E0E0 1px; } +.hexviewerwindow_border_middle {border-bottom: solid #E0E0E0 1px; border-top: solid #E0E0E0 1px;} +.hexviewerwindow_border_end {border-right: solid #E0E0E0 1px; border-top: solid #E0E0E0 1px; } diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-content-viewer/src/main/webapp/js/hexview/hexview.js b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-content-viewer/src/main/webapp/js/hexview/hexview.js new file mode 100644 index 0000000000..7c461d57f1 --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-content-viewer/src/main/webapp/js/hexview/hexview.js @@ -0,0 +1,199 @@ +$(document).ready(function () { + var HEX = '0123456789ABCDEF'; + + function dec2_to_hex(dec) + { + if (dec < 0) + dec = 0; + + if (dec > 255) + dec = 255; + + return HEX.charAt(Math.floor(dec / 16)) + HEX.charAt(dec % 16); + } + + function dec_to_hex8(dec) + { + var str = ""; + + for (var i = 3; i >= 0; i--) + { + str += dec2_to_hex((dec >> (i*8)) & 255); + } + + return str; + } + + function remove_whitespace(str) + { + return str.replace(/\n/g, "") + .replace(/\t/g, "") + .replace(/ /g, "") + .replace(/\r/g, ""); + } + + var BASE64_CHARS = "ABCDEFGHIJKLMNOPQRSTUVWXYZabcdefghijklmnopqrstuvwxyz0123456789+/="; + + function base64_decode(encoded) + { + var decoded = ""; + + for (var i = 0; i < encoded.length; i += 4) + { + var ch0 = encoded.charAt(i+0); + var ch1 = encoded.charAt(i+1); + var ch2 = encoded.charAt(i+2); + var ch3 = encoded.charAt(i+3); + + var index0 = BASE64_CHARS.indexOf(ch0); + var index1 = BASE64_CHARS.indexOf(ch1); + var index2 = BASE64_CHARS.indexOf(ch2); + var index3 = BASE64_CHARS.indexOf(ch3); + + decoded += String.fromCharCode((index0 << 2) | (index1 >> 4)); + decoded += String.fromCharCode(((index1 & 15) << 4) | (index2 >> 2)); + + // skip the base64 padding as those weren't present in the actual bytes + var token = String.fromCharCode(((index2 & 3) << 6) | index3); + if (index3 !== 64 || token !== '@') { + decoded += token; + } + } + + return decoded; + } + + function markup_hexviewwindow(div, index) + { + var entityMap = { + '&': '&', + '<': '<', + '>': '>', + '"': '"', + "'": ''', + '/': '/' + }; + + function escapeHtml(string) { + if (string === null || typeof string === 'undefined') { + return ''; + } else { + return String(string).replace(/[&<>"'\/]/g, function (s) { + return entityMap[s]; + }); + } + } + + var bin_data = base64_decode(remove_whitespace(div.text())); + var line_data; + var title = div.attr("title"); + + var highlights_str = $("form#hexviewwindow_params input[name='highlights']", div).attr("value").split(','); + var highlights = []; + + for (var i = 0; i < highlights_str.length; i++) + { + highlights.push(highlights_str[i].split(":")); + } + + var params = title.split(':'); + var step = parseInt($("form#hexviewwindow_params input[name='row_width']", div).attr("value")); + var word_size = parseInt($("form#hexviewwindow_params input[name='word_size']", div).attr("value")); + var hide_0x = parseInt($("form#hexviewwindow_params input[name='hide_0x']", div).attr("value")); + var decimal_offset = parseInt($("form#hexviewwindow_params input[name='decimal_offset']", div).attr("value")); + var start_byte_1 = parseInt($("form#hexviewwindow_params input[name='start_byte_1']", div).attr("value")); + var caption = $("form#hexviewwindow_params input[name='caption']", div).attr("value"); + + div.text(""); + div.append("
      "); + + var offset = (start_byte_1 ? 1 : 0); + + function apply_highlights(index) + { + for (var j = 0; j < highlights.length; j++) + { + if ((index >= highlights[j][0]) && (index <= highlights[j][1])) + { + if (index === highlights[j][0]) + { + $("table tr td:last", div).addClass("hexviewerwindow_border_start"); + } + + if (index === highlights[j][1]) + { + $("table tr td:last", div).addClass("hexviewerwindow_border_end"); + } + + $("table tr td:last", div).addClass("hexviewerwindow_code_hi hexviewerwindow_border_middle"); + $("table tr td:last", div).attr("style", "background-color: " + highlights[j][2] + ";"); + $("table tr td:last", div).attr("title", highlights[j][3]); + + runlen += 1; + } + else + { + $("table tr td:last", div).addClass("hexviewerwindow_code"); + } + } + } + + if (caption) + $("table", div).append("" + escapeHtml(caption) + ""); + + while (bin_data.length > 0) + { + line_data = bin_data.slice(0, step); + bin_data = bin_data.slice(step); + + $("table", div).addClass("hexviewerwindow_table"); + $("table", div).append("").addClass("hexviewerwindow"); + $("table tr:last", div).append("" + (decimal_offset ? ("00000000"+offset).slice(-8) : "0x" + dec_to_hex8(offset)) + ""); + $("table tr td:last", div).addClass("hexviewerwindow_offset"); + + var runlen = 0; + + for (var i = 0; i < line_data.length; i += word_size) + { + var num = ""; + + for (var j = 0; j < word_size; j++) + { + num += dec2_to_hex(line_data.charCodeAt(i+j)); + } + + $("table tr:last", div).append("" + (hide_0x ? "" : "0x") + num + ""); + + apply_highlights(offset+i); + } + + var text = ""; + + for (var i = 0; i < line_data.length; i++) + { + var cc = line_data.charCodeAt(i); + + if ((cc >= 32) && (cc <= 126)) + { + text = text + line_data.charAt(i); + } + else + { + text = text + "."; + } + } + + if (line_data.length < step) + $("table tr td:last", div).attr("colspan", Math.floor((step - line_data.length) / word_size) + 1); + + offset += step; + + $("table tr:last", div).append("" + escapeHtml(text) + ""); + $("table tr td:last", div).addClass("hexviewerwindow_visual"); + } + } + + $("div.hexviewwindow").each(function (index) { + markup_hexviewwindow($(this), index); + }); +}); \ No newline at end of file diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/web.xml b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/web.xml index eff7b8d37a..d0a5e39e80 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/web.xml +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/web.xml @@ -115,6 +115,17 @@ /bulletin-board + + + + MessagePage + /WEB-INF/pages/message-page.jsp + + + MessagePage + /message + + diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/codemirror/addon/fold/foldgutter.css b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/codemirror/addon/fold/foldgutter.css new file mode 100644 index 0000000000..ad19ae2d3e --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/codemirror/addon/fold/foldgutter.css @@ -0,0 +1,20 @@ +.CodeMirror-foldmarker { + color: blue; + text-shadow: #b9f 1px 1px 2px, #b9f -1px -1px 2px, #b9f 1px -1px 2px, #b9f -1px 1px 2px; + font-family: arial; + line-height: .3; + cursor: pointer; +} +.CodeMirror-foldgutter { + width: .7em; +} +.CodeMirror-foldgutter-open, +.CodeMirror-foldgutter-folded { + cursor: pointer; +} +.CodeMirror-foldgutter-open:after { + content: "\25BE"; +} +.CodeMirror-foldgutter-folded:after { + content: "\25B8"; +} diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/codemirror/lib/codemirror-compressed.js b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/codemirror/lib/codemirror-compressed.js index 11b3fbc5e5..262fea7a85 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/codemirror/lib/codemirror-compressed.js +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/codemirror/lib/codemirror-compressed.js @@ -1,16 +1,26 @@ /* CodeMirror - Minified & Bundled - Generated on 12/10/2014 with http://codemirror.net/doc/compress.html + Generated on 2/22/2015 with http://codemirror.net/doc/compress.html Version: 4.8 CodeMirror Library: - codemirror.js + Modes: + - javascript.js + - xml.js Add-ons: + - brace-fold.js + - comment-fold.js + - foldcode.js + - foldgutter.js + - markdown-fold.js - matchbrackets.js - show-hint.js + - xml-fold.js */ !function(a){if("object"==typeof exports&&"object"==typeof module)module.exports=a();else{if("function"==typeof define&&define.amd)return define([],a);this.CodeMirror=a()}}(function(){"use strict";function w(a,b){if(!(this instanceof w))return new w(a,b);this.options=b=b?Qg(b):{},Qg(ie,b,!1),J(b);var c=b.value;"string"==typeof c&&(c=new Lf(c,b.mode)),this.doc=c;var f=this.display=new x(a,c);f.wrapper.CodeMirror=this,F(this),D(this),b.lineWrapping&&(this.display.wrapper.className+=" CodeMirror-wrap"),b.autofocus&&!o&&_c(this),this.state={keyMaps:[],overlays:[],modeGen:0,overwrite:!1,focused:!1,suppressEdits:!1,pasteIncoming:!1,cutIncoming:!1,draggingText:!1,highlight:new Gg,keySeq:null},d&&11>e&&setTimeout(Rg($c,this,!0),20),cd(this),ih(),Ac(this),this.curOp.forceUpdate=!0,Pf(this,c),b.autofocus&&!o||bh()==f.input?setTimeout(Rg(Hd,this),20):Id(this);for(var g in je)je.hasOwnProperty(g)&&je[g](this,b[g],le);P(this);for(var h=0;he&&(c.gutters.style.zIndex=-1,c.scroller.style.paddingRight=0),n&&(g.style.width="0px"),f||(c.scroller.draggable=!0),k&&(c.inputDiv.style.height="1px",c.inputDiv.style.position="absolute"),d&&8>e&&(c.scrollbarH.style.minHeight=c.scrollbarV.style.minWidth="18px"),a&&(a.appendChild?a.appendChild(c.wrapper):a(c.wrapper)),c.viewFrom=c.viewTo=b.first,c.view=[],c.externalMeasured=null,c.viewOffset=0,c.lastWrapHeight=c.lastWrapWidth=0,c.updateLineNumbers=null,c.lineNumWidth=c.lineNumInnerWidth=c.lineNumChars=null,c.prevInput="",c.alignWidgets=!1,c.pollingFast=!1,c.poll=new Gg,c.cachedCharWidth=c.cachedTextHeight=c.cachedPaddingH=null,c.inaccurateSelection=!1,c.maxLine=null,c.maxLineLength=0,c.maxLineChanged=!1,c.wheelDX=c.wheelDY=c.wheelStartX=c.wheelStartY=null,c.shift=!1,c.selForContextMenu=null}function y(a){a.doc.mode=w.getMode(a.options,a.doc.modeOption),z(a)}function z(a){a.doc.iter(function(a){a.stateAfter&&(a.stateAfter=null),a.styles&&(a.styles=null)}),a.doc.frontier=a.doc.first,Tb(a,100),a.state.modeGen++,a.curOp&&Pc(a)}function A(a){a.options.lineWrapping?(eh(a.display.wrapper,"CodeMirror-wrap"),a.display.sizer.style.minWidth=""):(dh(a.display.wrapper,"CodeMirror-wrap"),I(a)),C(a),Pc(a),kc(a),setTimeout(function(){M(a)},100)}function B(a){var b=wc(a.display),c=a.options.lineWrapping,d=c&&Math.max(5,a.display.scroller.clientWidth/xc(a.display)-3);return function(e){if(ef(a.doc,e))return 0;var f=0;if(e.widgets)for(var g=0;gb.maxLineLength&&(b.maxLineLength=c,b.maxLine=a)})}function J(a){var b=Ng(a.gutters,"CodeMirror-linenumbers");-1==b&&a.lineNumbers?a.gutters=a.gutters.concat(["CodeMirror-linenumbers"]):b>-1&&!a.lineNumbers&&(a.gutters=a.gutters.slice(0),a.gutters.splice(b,1))}function K(a){return a.display.scroller.clientHeight-a.display.wrapper.clientHeightb.clientWidth;f&&b.scrollWidth<=b.clientWidth+1&&d>0&&!b.hScrollbarTakesSpace&&(f=!1);var g=e>b.clientHeight;if(g?(c.scrollbarV.style.display="block",c.scrollbarV.style.bottom=f?d+"px":"0",c.scrollbarV.firstChild.style.height=Math.max(0,e-b.clientHeight+(b.barHeight||c.scrollbarV.clientHeight))+"px"):(c.scrollbarV.style.display="",c.scrollbarV.firstChild.style.height="0"),f?(c.scrollbarH.style.display="block",c.scrollbarH.style.right=g?d+"px":"0",c.scrollbarH.firstChild.style.width=b.scrollWidth-b.clientWidth+(b.barWidth||c.scrollbarH.clientWidth)+"px"):(c.scrollbarH.style.display="",c.scrollbarH.firstChild.style.width="0"),f&&g?(c.scrollbarFiller.style.display="block",c.scrollbarFiller.style.height=c.scrollbarFiller.style.width=d+"px"):c.scrollbarFiller.style.display="",f&&a.options.coverGutterNextToScrollbar&&a.options.fixedGutter?(c.gutterFiller.style.display="block",c.gutterFiller.style.height=d+"px",c.gutterFiller.style.width=c.gutters.offsetWidth+"px"):c.gutterFiller.style.display="",!a.state.checkedOverlayScrollbar&&b.clientHeight>0){if(0===d){var h=p&&!l?"12px":"18px";c.scrollbarV.style.minWidth=c.scrollbarH.style.minHeight=h;var i=function(b){pg(b)!=c.scrollbarV&&pg(b)!=c.scrollbarH&&Kc(a,gd)(b)};rg(c.scrollbarV,"mousedown",i),rg(c.scrollbarH,"mousedown",i)}a.state.checkedOverlayScrollbar=!0}}function N(a,b,c){var d=c&&null!=c.top?Math.max(0,c.top):a.scroller.scrollTop;d=Math.floor(d-Xb(a));var e=c&&null!=c.bottom?c.bottom:d+a.wrapper.clientHeight,f=Vf(b,d),g=Vf(b,e);if(c&&c.ensure){var h=c.ensure.from.line,i=c.ensure.to.line;if(f>h)return{from:h,to:Vf(b,Wf(Qf(b,h))+a.wrapper.clientHeight)};if(Math.min(i,b.lastLine())>=g)return{from:Vf(b,Wf(Qf(b,i))-a.wrapper.clientHeight),to:i}}return{from:f,to:Math.max(g,f+1)}}function O(a){var b=a.display,c=b.view;if(b.alignWidgets||b.gutters.firstChild&&a.options.fixedGutter){for(var d=R(b)-b.scroller.scrollLeft+a.doc.scrollLeft,e=b.gutters.offsetWidth,f=d+"px",g=0;g=c.viewFrom&&b.visible.to<=c.viewTo&&(null==c.updateLineNumbers||c.updateLineNumbers>=c.viewTo)&&0==Vc(a))return!1;P(a)&&(Rc(a),b.dims=$(a));var e=d.first+d.size,f=Math.max(b.visible.from-a.options.viewportMargin,d.first),g=Math.min(e,b.visible.to+a.options.viewportMargin);c.viewFromg&&c.viewTo-g<20&&(g=Math.min(e,c.viewTo)),v&&(f=cf(a.doc,f),g=df(a.doc,g));var h=f!=c.viewFrom||g!=c.viewTo||c.lastWrapHeight!=b.wrapperHeight||c.lastWrapWidth!=b.wrapperWidth;Uc(a,f,g),c.viewOffset=Wf(Qf(a.doc,c.viewFrom)),a.display.mover.style.top=c.viewOffset+"px";var i=Vc(a);if(!h&&0==i&&!b.force&&(null==c.updateLineNumbers||c.updateLineNumbers>=c.viewTo))return!1;var j=bh();return i>4&&(c.lineDiv.style.display="none"),_(a,c.updateLineNumbers,b.dims),i>4&&(c.lineDiv.style.display=""),j&&bh()!=j&&j.offsetHeight&&j.focus(),$g(c.cursorDiv),$g(c.selectionDiv),h&&(c.lastWrapHeight=b.wrapperHeight,c.lastWrapWidth=b.wrapperWidth,Tb(a,400)),c.updateLineNumbers=null,!0}function U(a,b){for(var c=b.force,d=b.viewport,e=!0;;e=!1){if(e&&a.options.lineWrapping&&b.oldScrollerWidth!=a.display.scroller.clientWidth)c=!0;else if(c=!1,d&&null!=d.top&&(d={top:Math.min(a.doc.height+Yb(a.display)-Bg-a.display.scroller.clientHeight,d.top)}),b.visible=N(a.display,a.doc,d),b.visible.from>=a.display.viewFrom&&b.visible.to<=a.display.viewTo)break;if(!T(a,b))break;Y(a);var f=L(a);Pb(a),W(a,f),M(a,f)}vg(a,"update",a),(a.display.viewFrom!=b.oldViewFrom||a.display.viewTo!=b.oldViewTo)&&vg(a,"viewportChange",a,a.display.viewFrom,a.display.viewTo)}function V(a,b){var c=new S(a,b);if(T(a,c)){Y(a),U(a,c);var d=L(a);Pb(a),W(a,d),M(a,d)}}function W(a,b){a.display.sizer.style.minHeight=a.display.heightForcer.style.top=b.docHeight+"px",a.display.gutters.style.height=Math.max(b.docHeight,b.clientHeight-Bg)+"px"}function X(a,b){a.display.sizer.offsetWidth+a.display.gutters.offsetWidthe){var i=g.node.offsetTop+g.node.offsetHeight;h=i-c,c=i}else{var j=g.node.getBoundingClientRect();h=j.bottom-j.top}var k=g.line.height-h;if(2>h&&(h=wc(b)),(k>.001||-.001>k)&&(Tf(g.line,h),Z(g.line),g.rest))for(var l=0;l=b&&m.lineNumber;m.changes&&(Ng(m.changes,"gutter")>-1&&(o=!1),ab(a,m,k,c)),o&&($g(m.lineNumber),m.lineNumber.appendChild(document.createTextNode(Q(a.options,k)))),h=m.node.nextSibling}else{var n=ib(a,m,k,c);g.insertBefore(n,h)}k+=m.size}for(;h;)h=i(h)}function ab(a,b,c,d){for(var e=0;ee&&(a.node.style.zIndex=2)),a.node}function cb(a){var b=a.bgClass?a.bgClass+" "+(a.line.bgClass||""):a.line.bgClass;if(b&&(b+=" CodeMirror-linebackground"),a.background)b?a.background.className=b:(a.background.parentNode.removeChild(a.background),a.background=null);else if(b){var c=bb(a);a.background=c.insertBefore(Yg("div",null,b),c.firstChild)}}function db(a,b){var c=a.display.externalMeasured;return c&&c.line==b.line?(a.display.externalMeasured=null,b.measure=c.measure,c.built):zf(a,b)}function eb(a,b){var c=b.text.className,d=db(a,b);b.text==b.node&&(b.node=d.pre),b.text.parentNode.replaceChild(d.pre,b.text),b.text=d.pre,d.bgClass!=b.bgClass||d.textClass!=b.textClass?(b.bgClass=d.bgClass,b.textClass=d.textClass,fb(b)):c&&(b.text.className=c)}function fb(a){cb(a),a.line.wrapClass?bb(a).className=a.line.wrapClass:a.node!=a.text&&(a.node.className="");var b=a.textClass?a.textClass+" "+(a.line.textClass||""):a.line.textClass;a.text.className=b||""}function gb(a,b,c,d){b.gutter&&(b.node.removeChild(b.gutter),b.gutter=null);var e=b.line.gutterMarkers;if(a.options.lineNumbers||e){var f=bb(b),g=b.gutter=f.insertBefore(Yg("div",null,"CodeMirror-gutter-wrapper","left: "+(a.options.fixedGutter?d.fixedPos:-d.gutterTotalWidth)+"px; width: "+d.gutterTotalWidth+"px"),b.text);if(b.line.gutterClass&&(g.className+=" "+b.line.gutterClass),!a.options.lineNumbers||e&&e["CodeMirror-linenumbers"]||(b.lineNumber=g.appendChild(Yg("div",Q(a.options,c),"CodeMirror-linenumber CodeMirror-gutter-elt","left: "+d.gutterLeft["CodeMirror-linenumbers"]+"px; width: "+a.display.lineNumInnerWidth+"px"))),e)for(var h=0;h=0){var g=qb(f.from(),e.from()),h=pb(f.to(),e.to()),i=f.empty()?e.from()==e.head:f.from()==f.head;b>=d&&--b,a.splice(--d,2,new sb(i?h:g,i?g:h))}}return new rb(a,b)}function ub(a,b){return new rb([new sb(a,b||a)],0)}function vb(a,b){return Math.max(a.first,Math.min(b,a.first+a.size-1))}function wb(a,b){if(b.linec?mb(c,Qf(a,c).text.length):xb(b,Qf(a,b.line).text.length)}function xb(a,b){var c=a.ch;return null==c||c>b?mb(a.line,b):0>c?mb(a.line,0):a}function yb(a,b){return b>=a.first&&b=f.ch:j.to>f.ch))){if(d&&(tg(k,"beforeCursorEnter"),k.explicitlyCleared)){if(h.markedSpans){--i;continue}break}if(!k.atomic)continue;var l=k.find(0>g?-1:1);if(0==nb(l,f)&&(l.ch+=g,l.ch<0?l=l.line>a.first?wb(a,mb(l.line-1)):null:l.ch>h.text.length&&(l=l.lineb&&(b=0),b=Math.round(b),d=Math.round(d),f.appendChild(Yg("div",null,"CodeMirror-selected","position: absolute; left: "+a+"px; top: "+b+"px; width: "+(null==c?i-a:c)+"px; height: "+(d-b)+"px"))}function k(b,c,d){function m(c,d){return pc(a,mb(b,c),"div",f,d)}var k,l,f=Qf(e,b),g=f.text.length;return xh(Xf(f),c||0,null==d?g:d,function(a,b,e){var n,o,p,f=m(a,"left");if(a==b)n=f,o=p=f.left;else{if(n=m(b-1,"right"),"rtl"==e){var q=f;f=n,n=q}o=f.left,p=n.right}null==c&&0==a&&(o=h),n.top-f.top>3&&(j(o,f.top,null,f.bottom),o=h,f.bottoml.bottom||n.bottom==l.bottom&&n.right>l.right)&&(l=n),h+1>o&&(o=h),j(o,n.top,p-o,n.bottom)}),{start:k,end:l}}var d=a.display,e=a.doc,f=document.createDocumentFragment(),g=Zb(a.display),h=g.left,i=d.lineSpace.offsetWidth-g.right,l=b.from(),m=b.to();if(l.line==m.line)k(l.line,l.ch,m.ch);else{var n=Qf(e,l.line),o=Qf(e,m.line),p=af(n)==af(o),q=k(l.line,l.ch,p?n.text.length+1:null).end,r=k(m.line,p?0:null,m.ch).start;p&&(q.top0?b.blinker=setInterval(function(){b.cursorDiv.style.visibility=(c=!c)?"":"hidden"},a.options.cursorBlinkRate):a.options.cursorBlinkRate<0&&(b.cursorDiv.style.visibility="hidden")}}function Tb(a,b){a.doc.mode.startState&&a.doc.frontier=a.display.viewTo)){var c=+new Date+a.options.workTime,d=re(b.mode,Wb(a,b.frontier)),e=[];b.iter(b.frontier,Math.min(b.first+b.size,a.display.viewTo+500),function(f){if(b.frontier>=a.display.viewFrom){var g=f.styles,h=tf(a,f,d,!0);f.styles=h.styles;var i=f.styleClasses,j=h.classes;j?f.styleClasses=j:i&&(f.styleClasses=null);for(var k=!g||g.length!=f.styles.length||i!=j&&(!i||!j||i.bgClass!=j.bgClass||i.textClass!=j.textClass),l=0;!k&&lc?(Tb(a,a.options.workDelay),!0):void 0}),e.length&&Jc(a,function(){for(var b=0;bg;--h){if(h<=f.first)return f.first;var i=Qf(f,h-1);if(i.stateAfter&&(!c||h<=f.frontier))return h;var j=Hg(i.text,null,a.options.tabSize);(null==e||d>j)&&(e=h-1,d=j)}return e}function Wb(a,b,c){var d=a.doc,e=a.display;if(!d.mode.startState)return!0;var f=Vb(a,b,c),g=f>d.first&&Qf(d,f-1).stateAfter;return g=g?re(d.mode,g):se(d.mode),d.iter(f,b,function(c){vf(a,c.text,g);var h=f==b-1||0==f%5||f>=e.viewFrom&&f2&&f.push((i.bottom+j.top)/2-c.top)}}f.push(c.bottom-c.top)}}function _b(a,b,c){if(a.line==b)return{map:a.measure.map,cache:a.measure.cache};for(var d=0;dc)return{map:a.measure.maps[d],cache:a.measure.caches[d],before:!0}}function ac(a,b){b=af(b);var c=Uf(b),d=a.display.externalMeasured=new Nc(a.doc,b,c);d.lineN=c;var e=d.built=zf(a,d);return d.text=e.pre,_g(a.display.lineMeasure,e.pre),d}function bc(a,b,c,d){return ec(a,dc(a,b),c,d)}function cc(a,b){if(b>=a.display.viewFrom&&b=c.lineN&&bc?(i=0,j=1,k="left"):n>c?(i=c-m,j=i+1):(l==g.length-3||c==n&&g[l+3]>c)&&(j=n-m,i=j-1,c>=n&&(k="right")),null!=i){if(h=g[l+2],m==n&&f==(h.insertLeft?"left":"right")&&(k=f),"left"==f&&0==i)for(;l&&g[l-2]==g[l-3]&&g[l-1].insertLeft;)h=g[(l-=3)+2],k="left";if("right"==f&&i==n-m)for(;ll;l++){for(;i&&Xg(b.line.text.charAt(m+i));)--i;for(;n>m+j&&Xg(b.line.text.charAt(m+j));)++j;if(d&&9>e&&0==i&&j==n-m)o=h.parentNode.getBoundingClientRect();else if(d&&a.options.lineWrapping){var p=Zg(h,i,j).getClientRects();o=p.length?p["right"==f?p.length-1:0]:fc}else o=Zg(h,i,j).getBoundingClientRect()||fc;if(o.left||o.right||0==i)break;j=i,i-=1,k="right"}d&&11>e&&(o=hc(a.display.measure,o))}else{i>0&&(k=f="right");var p;o=a.options.lineWrapping&&(p=h.getClientRects()).length>1?p["right"==f?p.length-1:0]:h.getBoundingClientRect()}if(d&&9>e&&!i&&(!o||!o.left&&!o.right)){var q=h.parentNode.getClientRects()[0];o=q?{left:q.left,right:q.left+xc(a.display),top:q.top,bottom:q.bottom}:fc}for(var r=o.top-b.rect.top,s=o.bottom-b.rect.top,t=(r+s)/2,u=b.view.measure.heights,l=0;lc.from?g(a-1):g(a,d)}d=d||Qf(a.doc,b.line),e||(e=dc(a,d));var i=Xf(d),j=b.ch;if(!i)return g(j);var k=Hh(i,j),l=h(j,k);return null!=Gh&&(l.other=h(j,Gh)),l}function rc(a,b){var c=0,b=wb(a.doc,b);a.options.lineWrapping||(c=xc(a.display)*b.ch);var d=Qf(a.doc,b.line),e=Wf(d)+Xb(a.display);return{left:c,right:c,top:e,bottom:e+d.height}}function sc(a,b,c,d){var e=mb(a,b);return e.xRel=d,c&&(e.outside=!0),e}function tc(a,b,c){var d=a.doc;if(c+=a.display.viewOffset,0>c)return sc(d.first,0,!0,-1);var e=Vf(d,c),f=d.first+d.size-1;if(e>f)return sc(d.first+d.size-1,Qf(d,f).text.length,!0,1);0>b&&(b=0);for(var g=Qf(d,e);;){var h=uc(a,g,e,b,c),i=$e(g),j=i&&i.find(0,!0);if(!i||!(h.ch>j.from.ch||h.ch==j.from.ch&&h.xRel>0))return h;e=Uf(g=j.to.line)}}function uc(a,b,c,d,e){function j(d){var e=qc(a,mb(c,d),"line",b,i);return g=!0,f>e.bottom?e.left-h:fq)return sc(c,n,r,1);for(;;){if(k?n==m||n==Jh(b,m,1):1>=n-m){for(var s=o>d||q-d>=d-o?m:n,t=d-(s==m?o:q);Xg(b.text.charAt(s));)++s;var u=sc(c,s,s==m?p:r,-1>t?-1:t>1?1:0);return u}var v=Math.ceil(l/2),w=m+v;if(k){w=m;for(var x=0;v>x;++x)w=Jh(b,w,1)}var y=j(w);y>d?(n=w,q=y,(r=g)&&(q+=1e3),l=v):(m=w,o=y,p=g,l-=v)}}function wc(a){if(null!=a.cachedTextHeight)return a.cachedTextHeight;if(null==vc){vc=Yg("pre");for(var b=0;49>b;++b)vc.appendChild(document.createTextNode("x")),vc.appendChild(Yg("br"));vc.appendChild(document.createTextNode("x"))}_g(a.measure,vc);var c=vc.offsetHeight/50;return c>3&&(a.cachedTextHeight=c),$g(a.measure),c||1}function xc(a){if(null!=a.cachedCharWidth)return a.cachedCharWidth;var b=Yg("span","xxxxxxxxxx"),c=Yg("pre",[b]);_g(a.measure,c);var d=b.getBoundingClientRect(),e=(d.right-d.left)/10;return e>2&&(a.cachedCharWidth=e),e||10}function Ac(a){a.curOp={cm:a,viewChanged:!1,startHeight:a.doc.height,forceUpdate:!1,updateInput:null,typing:!1,changeObjs:null,cursorActivityHandlers:null,cursorActivityCalled:0,selectionChanged:!1,updateMaxLine:!1,scrollLeft:null,scrollTop:null,scrollToPos:null,id:++zc},yc?yc.ops.push(a.curOp):a.curOp.ownsGroup=yc={ops:[a.curOp],delayedCallbacks:[]}}function Bc(a){var b=a.delayedCallbacks,c=0;do{for(;c=c.viewTo)||c.maxLineChanged&&b.options.lineWrapping,a.update=a.mustUpdate&&new S(b,a.mustUpdate&&{top:a.scrollTop,ensure:a.scrollToPos},a.forceUpdate)}function Fc(a){a.updatedDisplay=a.mustUpdate&&T(a.cm,a.update)}function Gc(a){var b=a.cm,c=b.display;a.updatedDisplay&&Y(b),a.barMeasure=L(b),c.maxLineChanged&&!b.options.lineWrapping&&(a.adjustWidthTo=bc(b,c.maxLine,c.maxLine.text.length).left+3,a.maxScrollLeft=Math.max(0,c.sizer.offsetLeft+a.adjustWidthTo+Bg-c.scroller.clientWidth)),(a.updatedDisplay||a.selectionChanged)&&(a.newSelectionNodes=Nb(b))}function Hc(a){var b=a.cm;null!=a.adjustWidthTo&&(b.display.sizer.style.minWidth=a.adjustWidthTo+"px",a.maxScrollLeft1&&M(b),a.updatedDisplay&&U(b,a.update),null==c.wheelStartX||null==a.scrollTop&&null==a.scrollLeft&&!a.scrollToPos||(c.wheelStartX=c.wheelStartY=null),null!=a.scrollTop&&(c.scroller.scrollTop!=a.scrollTop||a.forceScroll)){var e=Math.max(0,Math.min(c.scroller.scrollHeight-c.scroller.clientHeight,a.scrollTop));c.scroller.scrollTop=c.scrollbarV.scrollTop=d.scrollTop=e}if(null!=a.scrollLeft&&(c.scroller.scrollLeft!=a.scrollLeft||a.forceScroll)){var g=Math.max(0,Math.min(c.scroller.scrollWidth-c.scroller.clientWidth,a.scrollLeft));c.scroller.scrollLeft=c.scrollbarH.scrollLeft=d.scrollLeft=g,O(b)}if(a.scrollToPos){var h=Zd(b,wb(d,a.scrollToPos.from),wb(d,a.scrollToPos.to),a.scrollToPos.margin);a.scrollToPos.isCursor&&b.state.focused&&Yd(b,h)}var i=a.maybeHiddenMarkers,j=a.maybeUnhiddenMarkers;if(i)for(var k=0;ka.barMeasure.clientWidth&&a.barMeasure.scrollWidthf;f=e){var g=new Nc(a.doc,Qf(a.doc,f),f);e=f+g.size,d.push(g)}return d}function Pc(a,b,c,d){null==b&&(b=a.doc.first),null==c&&(c=a.doc.first+a.doc.size),d||(d=0);var e=a.display;if(d&&cb)&&(e.updateLineNumbers=b),a.curOp.viewChanged=!0,b>=e.viewTo)v&&cf(a.doc,b)e.viewFrom?Rc(a):(e.viewFrom+=d,e.viewTo+=d);else if(b<=e.viewFrom&&c>=e.viewTo)Rc(a);else if(b<=e.viewFrom){var f=Tc(a,c,c+d,1);f?(e.view=e.view.slice(f.index),e.viewFrom=f.lineN,e.viewTo+=d):Rc(a)}else if(c>=e.viewTo){var f=Tc(a,b,b,-1);f?(e.view=e.view.slice(0,f.index),e.viewTo=f.lineN):Rc(a)}else{var g=Tc(a,b,b,-1),h=Tc(a,c,c+d,1);g&&h?(e.view=e.view.slice(0,g.index).concat(Oc(a,g.lineN,h.lineN)).concat(e.view.slice(h.index)),e.viewTo+=d):Rc(a)}var i=e.externalMeasured;i&&(c=e.lineN&&b=d.viewTo)){var f=d.view[Sc(a,b)];if(null!=f.node){var g=f.changes||(f.changes=[]);-1==Ng(g,c)&&g.push(c)}}}function Rc(a){a.display.viewFrom=a.display.viewTo=a.doc.first,a.display.view=[],a.display.viewOffset=0}function Sc(a,b){if(b>=a.display.viewTo)return null;if(b-=a.display.viewFrom,0>b)return null;for(var c=a.display.view,d=0;db)return d}function Tc(a,b,c,d){var f,e=Sc(a,b),g=a.display.view;if(!v||c==a.doc.first+a.doc.size)return{index:e,lineN:c};for(var h=0,i=a.display.viewFrom;e>h;h++)i+=g[h].size;if(i!=b){if(d>0){if(e==g.length-1)return null;f=i+g[e].size-b,e++}else f=i-b;b+=f,c+=f}for(;cf(a.doc,c)!=c;){if(e==(0>d?0:g.length-1))return null;c+=d*g[e-(0>d?1:0)].size,e+=d}return{index:e,lineN:c}}function Uc(a,b,c){var d=a.display,e=d.view;0==e.length||b>=d.viewTo||c<=d.viewFrom?(d.view=Oc(a,b,c),d.viewFrom=b):(d.viewFrom>b?d.view=Oc(a,b,d.viewFrom).concat(d.view):d.viewFromc&&(d.view=d.view.slice(0,Sc(a,c)))),d.viewTo=c}function Vc(a){for(var b=a.display.view,c=0,d=0;d=9&&a.display.inputHasSelection===g||p&&/[\uf700-\uf7ff]/.test(g))return $c(a),!1;var h=!a.curOp;h&&Ac(a),a.display.shift=!1,8203!=g.charCodeAt(0)||f.sel!=a.display.selForContextMenu||c||(c="\u200b");for(var i=0,j=Math.min(c.length,g.length);j>i&&c.charCodeAt(i)==g.charCodeAt(i);)++i;var k=g.slice(i),l=rh(k),m=null;a.state.pasteIncoming&&f.sel.ranges.length>1&&(Yc&&Yc.join("\n")==k?m=0==f.sel.ranges.length%Yc.length&&Og(Yc,rh):l.length==f.sel.ranges.length&&(m=Og(l,function(a){return[a]})));for(var n=f.sel.ranges.length-1;n>=0;n--){var o=f.sel.ranges[n],q=o.from(),r=o.to();i-1){de(a,v.line,"smart");break}}else u.electricInput&&u.electricInput.test(Qf(f,v.line).text.slice(0,v.ch))&&de(a,v.line,"smart")}}return be(a),a.curOp.updateInput=s,a.curOp.typing=!0,g.length>1e3||g.indexOf("\n")>-1?b.value=a.display.prevInput="":a.display.prevInput=g,h&&Cc(a),a.state.pasteIncoming=a.state.cutIncoming=!1,!0}function $c(a,b){var c,f,g=a.doc;if(a.somethingSelected()){a.display.prevInput="";var h=g.sel.primary();c=th&&(h.to().line-h.from().line>100||(f=a.getSelection()).length>1e3);var i=c?"-":f||a.getSelection();a.display.input.value=i,a.state.focused&&Mg(a.display.input),d&&e>=9&&(a.display.inputHasSelection=i)}else b||(a.display.prevInput=a.display.input.value="",d&&e>=9&&(a.display.inputHasSelection=null));a.display.inaccurateSelection=c}function _c(a){"nocursor"==a.options.readOnly||o&&bh()==a.display.input||a.display.input.focus()}function ad(a){a.state.focused||(_c(a),Hd(a))}function bd(a){return a.options.readOnly||a.doc.cantEdit}function cd(a){function c(){a.state.focused&&setTimeout(Rg(_c,a),0)}function g(b){xg(a,b)||og(b)}function h(c){if(a.somethingSelected())Yc=a.getSelections(),b.inaccurateSelection&&(b.prevInput="",b.inaccurateSelection=!1,b.input.value=Yc.join("\n"),Mg(b.input));else{for(var d=[],e=[],f=0;fe?rg(b.scroller,"dblclick",Kc(a,function(b){if(!xg(a,b)){var c=fd(a,b);if(c&&!nd(a,b)&&!ed(a.display,b)){lg(b);var d=a.findWordAt(c);Bb(a.doc,d.anchor,d.head)}}})):rg(b.scroller,"dblclick",function(b){xg(a,b)||lg(b)}),rg(b.lineSpace,"selectstart",function(a){ed(b,a)||lg(a)}),t||rg(b.scroller,"contextmenu",function(b){Jd(a,b)}),rg(b.scroller,"scroll",function(){b.scroller.clientHeight&&(rd(a,b.scroller.scrollTop),sd(a,b.scroller.scrollLeft,!0),tg(a,"scroll",a))}),rg(b.scrollbarV,"scroll",function(){b.scroller.clientHeight&&rd(a,b.scrollbarV.scrollTop)}),rg(b.scrollbarH,"scroll",function(){b.scroller.clientHeight&&sd(a,b.scrollbarH.scrollLeft)}),rg(b.scroller,"mousewheel",function(b){vd(a,b)}),rg(b.scroller,"DOMMouseScroll",function(b){vd(a,b)}),rg(b.scrollbarH,"mousedown",c),rg(b.scrollbarV,"mousedown",c),rg(b.wrapper,"scroll",function(){b.wrapper.scrollTop=b.wrapper.scrollLeft=0}),rg(b.input,"keyup",function(b){Fd.call(a,b)}),rg(b.input,"input",function(){d&&e>=9&&a.display.inputHasSelection&&(a.display.inputHasSelection=null),Xc(a)}),rg(b.input,"keydown",Kc(a,Dd)),rg(b.input,"keypress",Kc(a,Gd)),rg(b.input,"focus",Rg(Hd,a)),rg(b.input,"blur",Rg(Id,a)),a.options.dragDrop&&(rg(b.scroller,"dragstart",function(b){qd(a,b)}),rg(b.scroller,"dragenter",g),rg(b.scroller,"dragover",g),rg(b.scroller,"drop",Kc(a,pd))),rg(b.scroller,"paste",function(c){ed(b,c)||(a.state.pasteIncoming=!0,_c(a),Xc(a))}),rg(b.input,"paste",function(){if(f&&!a.state.fakedLastChar&&!(new Date-a.state.lastMiddleDown<200)){var c=b.input.selectionStart,d=b.input.selectionEnd;b.input.value+="$",b.input.selectionEnd=d,b.input.selectionStart=c,a.state.fakedLastChar=!0}a.state.pasteIncoming=!0,Xc(a)}),rg(b.input,"cut",h),rg(b.input,"copy",h),k&&rg(b.sizer,"mouseup",function(){bh()==b.input&&b.input.blur(),_c(a)})}function dd(a){var b=a.display;(b.lastWrapHeight!=b.wrapper.clientHeight||b.lastWrapWidth!=b.wrapper.clientWidth)&&(b.cachedCharWidth=b.cachedTextHeight=b.cachedPaddingH=null,a.setSize())}function ed(a,b){for(var c=pg(b);c!=a.wrapper;c=c.parentNode)if(!c||c.ignoreEvents||c.parentNode==a.sizer&&c!=a.mover)return!0}function fd(a,b,c,d){var e=a.display;if(!c){var f=pg(b);if(f==e.scrollbarH||f==e.scrollbarV||f==e.scrollbarFiller||f==e.gutterFiller)return null}var g,h,i=e.lineSpace.getBoundingClientRect();try{g=b.clientX-i.left,h=b.clientY-i.top}catch(b){return null}var k,j=tc(a,g,h);if(d&&1==j.xRel&&(k=Qf(a.doc,j.line).text).length==j.ch){var l=Hg(k,k.length,a.options.tabSize)-k.length;j=mb(j.line,Math.max(0,Math.round((g-Zb(a.display).left)/xc(a.display))-l))}return j}function gd(a){if(!xg(this,a)){var b=this,c=b.display;if(c.shift=a.shiftKey,ed(c,a))return f||(c.scroller.draggable=!1,setTimeout(function(){c.scroller.draggable=!0},100)),void 0;if(!nd(b,a)){var d=fd(b,a);switch(window.focus(),qg(a)){case 1:d?jd(b,a,d):pg(a)==c.scroller&&lg(a);break;case 2:f&&(b.state.lastMiddleDown=+new Date),d&&Bb(b.doc,d),setTimeout(Rg(_c,b),20),lg(a);break;case 3:t&&Jd(b,a)}}}}function jd(a,b,c){setTimeout(Rg(ad,a),0);var e,d=+new Date;id&&id.time>d-400&&0==nb(id.pos,c)?e="triple":hd&&hd.time>d-400&&0==nb(hd.pos,c)?(e="double",id={time:d,pos:c}):(e="single",hd={time:d,pos:c});var f=a.doc.sel,g=p?b.metaKey:b.ctrlKey;a.options.dragDrop&&kh&&!bd(a)&&"single"==e&&f.contains(c)>-1&&f.somethingSelected()?kd(a,b,c,g):ld(a,b,c,e,g)}function kd(a,b,c,g){var h=a.display,i=Kc(a,function(j){f&&(h.scroller.draggable=!1),a.state.draggingText=!1,sg(document,"mouseup",i),sg(h.scroller,"drop",i),Math.abs(b.clientX-j.clientX)+Math.abs(b.clientY-j.clientY)<10&&(lg(j),g||Bb(a.doc,c),_c(a),d&&9==e&&setTimeout(function(){document.body.focus(),_c(a)},20))});f&&(h.scroller.draggable=!0),a.state.draggingText=i,h.scroller.dragDrop&&h.scroller.dragDrop(),rg(document,"mouseup",i),rg(h.scroller,"drop",i)}function ld(a,b,c,d,e){function n(b){if(0!=nb(m,b))if(m=b,"rect"==d){for(var e=[],f=a.options.tabSize,k=Hg(Qf(g,c.line).text,c.ch,f),l=Hg(Qf(g,b.line).text,b.ch,f),n=Math.min(k,l),o=Math.max(k,l),p=Math.min(c.line,b.line),q=Math.min(a.lastLine(),Math.max(c.line,b.line));q>=p;p++){var r=Qf(g,p).text,s=Ig(r,n,f);n==o?e.push(new sb(mb(p,s),mb(p,s))):r.length>s&&e.push(new sb(mb(p,s),mb(p,Ig(r,o,f))))}e.length||e.push(new sb(c,c)),Hb(g,tb(j.ranges.slice(0,i).concat(e),i),{origin:"*mouse",scroll:!1}),a.scrollIntoView(b)}else{var t=h,u=t.anchor,v=b;if("single"!=d){if("double"==d)var w=a.findWordAt(b);else var w=new sb(mb(b.line,0),wb(g,mb(b.line+1,0)));nb(w.anchor,u)>0?(v=w.head,u=qb(t.from(),w.anchor)):(v=w.anchor,u=pb(t.to(),w.head))}var e=j.ranges.slice(0);e[i]=new sb(wb(g,u),v),Hb(g,tb(e,i),Eg)}}function q(b){var c=++p,e=fd(a,b,!0,"rect"==d);if(e)if(0!=nb(e,m)){ad(a),n(e);var h=N(f,g);(e.line>=h.to||e.lineo.bottom?20:0;i&&setTimeout(Kc(a,function(){p==c&&(f.scroller.scrollTop+=i,q(b))}),50)}}function r(b){p=1/0,lg(b),_c(a),sg(document,"mousemove",s),sg(document,"mouseup",t),g.history.lastSelOrigin=null}var f=a.display,g=a.doc;lg(b);var h,i,j=g.sel;if(e&&!b.shiftKey?(i=g.sel.contains(c),h=i>-1?g.sel.ranges[i]:new sb(c,c)):h=g.sel.primary(),b.altKey)d="rect",e||(h=new sb(c,c)),c=fd(a,b,!0,!0),i=-1;else if("double"==d){var k=a.findWordAt(c);h=a.display.shift||g.extend?Ab(g,h,k.anchor,k.head):k}else if("triple"==d){var l=new sb(mb(c.line,0),wb(g,mb(c.line+1,0)));h=a.display.shift||g.extend?Ab(g,h,l.anchor,l.head):l}else h=Ab(g,h,c);e?i>-1?Db(g,i,h,Eg):(i=g.sel.ranges.length,Hb(g,tb(g.sel.ranges.concat([h]),i),{scroll:!1,origin:"*mouse"})):(i=0,Hb(g,new rb([h],0),Eg),j=g.sel);var m=c,o=f.wrapper.getBoundingClientRect(),p=0,s=Kc(a,function(a){qg(a)?q(a):r(a)}),t=Kc(a,r);rg(document,"mousemove",s),rg(document,"mouseup",t)}function md(a,b,c,d,e){try{var f=b.clientX,g=b.clientY}catch(b){return!1}if(f>=Math.floor(a.display.gutters.getBoundingClientRect().right))return!1;d&&lg(b);var h=a.display,i=h.lineDiv.getBoundingClientRect();if(g>i.bottom||!zg(a,c))return ng(b);g-=i.top-h.viewOffset;for(var j=0;j=f){var l=Vf(a.doc,g),m=a.options.gutters[j];return e(a,c,a,l,m,b),ng(b)}}}function nd(a,b){return md(a,b,"gutterClick",!0,vg)}function pd(a){var b=this;if(!xg(b,a)&&!ed(b.display,a)){lg(a),d&&(od=+new Date);var c=fd(b,a,!0),e=a.dataTransfer.files;if(c&&!bd(b))if(e&&e.length&&window.FileReader&&window.File)for(var f=e.length,g=Array(f),h=0,i=function(a,d){var e=new FileReader;e.onload=Kc(b,function(){if(g[d]=e.result,++h==f){c=wb(b.doc,c);var a={from:c,to:c,text:rh(g.join("\n")),origin:"paste"};Rd(b.doc,a),Gb(b.doc,ub(c,Ld(a)))}}),e.readAsText(a)},j=0;f>j;++j)i(e[j],j);else{if(b.state.draggingText&&b.doc.sel.contains(c)>-1)return b.state.draggingText(a),setTimeout(Rg(_c,b),20),void 0;try{var g=a.dataTransfer.getData("Text");if(g){if(b.state.draggingText&&!(p?a.metaKey:a.ctrlKey))var k=b.listSelections();if(Ib(b.doc,ub(c,c)),k)for(var j=0;jh.clientWidth||e&&h.scrollHeight>h.clientHeight){if(e&&p&&f)a:for(var j=c.target,k=g.view;j!=h;j=j.parentNode)for(var l=0;lm?n=Math.max(0,n+m-50):o=Math.min(b.doc.height,o+m+50),V(b,{top:n,bottom:o})}20>td&&(null==g.wheelStartX?(g.wheelStartX=h.scrollLeft,g.wheelStartY=h.scrollTop,g.wheelDX=d,g.wheelDY=e,setTimeout(function(){if(null!=g.wheelStartX){var a=h.scrollLeft-g.wheelStartX,b=h.scrollTop-g.wheelStartY,c=b&&g.wheelDY&&b/g.wheelDY||a&&g.wheelDX&&a/g.wheelDX;g.wheelStartX=g.wheelStartY=null,c&&(ud=(ud*td+c)/(td+1),++td)}},200)):(g.wheelDX+=d,g.wheelDY+=e))}}function wd(a,b,c){if("string"==typeof b&&(b=te[b],!b))return!1;a.display.pollingFast&&Zc(a)&&(a.display.pollingFast=!1);var d=a.display.shift,e=!1;try{bd(a)&&(a.state.suppressEdits=!0),c&&(a.display.shift=!1),e=b(a)!=Cg}finally{a.display.shift=d,a.state.suppressEdits=!1}return e}function xd(a,b,c){for(var d=0;de&&27==a.keyCode&&(a.returnValue=!1);var c=a.keyCode;b.display.shift=16==c||a.shiftKey;var f=Ad(b,a);i&&(Cd=f?c:null,!f&&88==c&&!th&&(p?a.metaKey:a.ctrlKey)&&b.replaceSelection("",null,"cut")),18!=c||/\bCodeMirror-crosshair\b/.test(b.display.lineDiv.className)||Ed(b)}}function Ed(a){function c(a){18!=a.keyCode&&a.altKey||(dh(b,"CodeMirror-crosshair"),sg(document,"keyup",c),sg(document,"mouseover",c))}var b=a.display.lineDiv;eh(b,"CodeMirror-crosshair"),rg(document,"keyup",c),rg(document,"mouseover",c)}function Fd(a){16==a.keyCode&&(this.doc.sel.shift=!1),xg(this,a)}function Gd(a){var b=this;if(!(xg(b,a)||a.ctrlKey&&!a.altKey||p&&a.metaKey)){var c=a.keyCode,f=a.charCode;if(i&&c==Cd)return Cd=null,lg(a),void 0;if(!(i&&(!a.which||a.which<10)||k)||!Ad(b,a)){var g=String.fromCharCode(null==f?c:f);Bd(b,a,g)||(d&&e>=9&&(b.display.inputHasSelection=null),Xc(b))}}}function Hd(a){"nocursor"!=a.options.readOnly&&(a.state.focused||(tg(a,"focus",a),a.state.focused=!0,eh(a.display.wrapper,"CodeMirror-focused"),a.curOp||a.display.selForContextMenu==a.doc.sel||($c(a),f&&setTimeout(Rg($c,a,!0),0))),Wc(a),Sb(a))}function Id(a){a.state.focused&&(tg(a,"blur",a),a.state.focused=!1,dh(a.display.wrapper,"CodeMirror-focused")),clearInterval(a.display.blinker),setTimeout(function(){a.state.focused||(a.display.shift=!1)},150)}function Jd(a,b){function m(){if(null!=c.input.selectionStart){var b=a.somethingSelected(),d=c.input.value="\u200b"+(b?c.input.value:"");c.prevInput=b?"":"\u200b",c.input.selectionStart=1,c.input.selectionEnd=d.length,c.selForContextMenu=a.doc.sel}}function n(){if(c.inputDiv.style.position="relative",c.input.style.cssText=k,d&&9>e&&(c.scrollbarV.scrollTop=c.scroller.scrollTop=h),Wc(a),null!=c.input.selectionStart){(!d||d&&9>e)&&m();var b=0,f=function(){c.selForContextMenu==a.doc.sel&&0==c.input.selectionStart?Kc(a,te.selectAll)(a):b++<10?c.detectingSelectAll=setTimeout(f,500):$c(a)};c.detectingSelectAll=setTimeout(f,200)}}if(!xg(a,b,"contextmenu")){var c=a.display;if(!ed(c,b)&&!Kd(a,b)){var g=fd(a,b),h=c.scroller.scrollTop;if(g&&!i){var j=a.options.resetSelectionOnContextMenu;j&&-1==a.doc.sel.contains(g)&&Kc(a,Hb)(a.doc,ub(g),Dg);var k=c.input.style.cssText;if(c.inputDiv.style.position="absolute",c.input.style.cssText="position: fixed; width: 30px; height: 30px; top: "+(b.clientY-5)+"px; left: "+(b.clientX-5)+"px; z-index: 1000; background: "+(d?"rgba(255, 255, 255, .05)":"transparent")+"; outline: none; border-width: 0; outline: none; overflow: hidden; opacity: .05; filter: alpha(opacity=5);",f)var l=window.scrollY;if(_c(a),f&&window.scrollTo(null,l),$c(a),a.somethingSelected()||(c.input.value=c.prevInput=" "),c.selForContextMenu=a.doc.sel,clearTimeout(c.detectingSelectAll),d&&e>=9&&m(),t){og(b);var o=function(){sg(window,"mouseup",o),setTimeout(n,20)};rg(window,"mouseup",o)}else setTimeout(n,50)}}}}function Kd(a,b){return zg(a,"gutterContextMenu")?md(a,b,"gutterContextMenu",!1,tg):!1}function Md(a,b){if(nb(a,b.from)<0)return a;if(nb(a,b.to)<=0)return Ld(b);var c=a.line+b.text.length-(b.to.line-b.from.line)-1,d=a.ch;return a.line==b.to.line&&(d+=Ld(b).ch-b.to.ch),mb(c,d)}function Nd(a,b){for(var c=[],d=0;d=0;--e)Sd(a,{from:d[e].from,to:d[e].to,text:e?[""]:b.text});else Sd(a,b)}}function Sd(a,b){if(1!=b.text.length||""!=b.text[0]||0!=nb(b.from,b.to)){var c=Nd(a,b);ag(a,b,c,a.cm?a.cm.curOp.id:0/0),Vd(a,b,c,Pe(a,b));var d=[];Of(a,function(a,c){c||-1!=Ng(d,a.history)||(kg(a.history,b),d.push(a.history)),Vd(a,b,null,Pe(a,b))})}}function Td(a,b,c){if(!a.cm||!a.cm.state.suppressEdits){for(var e,d=a.history,f=a.sel,g="undo"==b?d.done:d.undone,h="undo"==b?d.undone:d.done,i=0;i=0;--i){var l=e.changes[i];if(l.origin=b,k&&!Qd(a,l,!1))return g.length=0,void 0;j.push(Zf(a,l));var m=i?Nd(a,l):Lg(g);Vd(a,l,m,Re(a,l)),!i&&a.cm&&a.cm.scrollIntoView({from:l.from,to:Ld(l)});var n=[];Of(a,function(a,b){b||-1!=Ng(n,a.history)||(kg(a.history,l),n.push(a.history)),Vd(a,l,null,Re(a,l))})}}}}function Ud(a,b){if(0!=b&&(a.first+=b,a.sel=new rb(Og(a.sel.ranges,function(a){return new sb(mb(a.anchor.line+b,a.anchor.ch),mb(a.head.line+b,a.head.ch))}),a.sel.primIndex),a.cm)){Pc(a.cm,a.first,a.first-b,b);for(var c=a.cm.display,d=c.viewFrom;da.lastLine())){if(b.from.linef&&(b={from:b.from,to:mb(f,Qf(a,f).text.length),text:[b.text[0]],origin:b.origin}),b.removed=Rf(a,b.from,b.to),c||(c=Nd(a,b)),a.cm?Wd(a.cm,b,d):Hf(a,b,d),Ib(a,c,Dg)}}function Wd(a,b,c){var d=a.doc,e=a.display,f=b.from,g=b.to,h=!1,i=f.line;a.options.lineWrapping||(i=Uf(af(Qf(d,f.line))),d.iter(i,g.line+1,function(a){return a==e.maxLine?(h=!0,!0):void 0})),d.sel.contains(b.from,b.to)>-1&&yg(a),Hf(d,b,c,B(a)),a.options.lineWrapping||(d.iter(i,f.line+b.text.length,function(a){var b=H(a);b>e.maxLineLength&&(e.maxLine=a,e.maxLineLength=b,e.maxLineChanged=!0,h=!1)}),h&&(a.curOp.updateMaxLine=!0)),d.frontier=Math.min(d.frontier,f.line),Tb(a,400);var j=b.text.length-(g.line-f.line)-1;f.line!=g.line||1!=b.text.length||Gf(a.doc,b)?Pc(a,f.line,g.line+1,j):Qc(a,f.line,"text");var k=zg(a,"changes"),l=zg(a,"change");if(l||k){var m={from:f,to:g,text:b.text,removed:b.removed,origin:b.origin};l&&vg(a,"change",a,m),k&&(a.curOp.changeObjs||(a.curOp.changeObjs=[])).push(m)}a.display.selForContextMenu=null}function Xd(a,b,c,d,e){if(d||(d=c),nb(d,c)<0){var f=d;d=c,c=f}"string"==typeof b&&(b=rh(b)),Rd(a,{from:c,to:d,text:b,origin:e})}function Yd(a,b){if(!xg(a,"scrollCursorIntoView")){var c=a.display,d=c.sizer.getBoundingClientRect(),e=null;if(b.top+d.top<0?e=!0:b.bottom+d.top>(window.innerHeight||document.documentElement.clientHeight)&&(e=!1),null!=e&&!m){var f=Yg("div","\u200b",null,"position: absolute; top: "+(b.top-c.viewOffset-Xb(a.display))+"px; height: "+(b.bottom-b.top+Bg)+"px; left: "+b.left+"px; width: 2px;");a.display.lineSpace.appendChild(f),f.scrollIntoView(e),a.display.lineSpace.removeChild(f)}}}function Zd(a,b,c,d){null==d&&(d=0);for(var e=0;5>e;e++){var f=!1,g=qc(a,b),h=c&&c!=b?qc(a,c):g,i=_d(a,Math.min(g.left,h.left),Math.min(g.top,h.top)-d,Math.max(g.left,h.left),Math.max(g.bottom,h.bottom)+d),j=a.doc.scrollTop,k=a.doc.scrollLeft;if(null!=i.scrollTop&&(rd(a,i.scrollTop),Math.abs(a.doc.scrollTop-j)>1&&(f=!0)),null!=i.scrollLeft&&(sd(a,i.scrollLeft),Math.abs(a.doc.scrollLeft-k)>1&&(f=!0)),!f)return g}}function $d(a,b,c,d,e){var f=_d(a,b,c,d,e);null!=f.scrollTop&&rd(a,f.scrollTop),null!=f.scrollLeft&&sd(a,f.scrollLeft)}function _d(a,b,c,d,e){var f=a.display,g=wc(a.display);0>c&&(c=0);var h=a.curOp&&null!=a.curOp.scrollTop?a.curOp.scrollTop:f.scroller.scrollTop,i=f.scroller.clientHeight-Bg,j={};e-c>i&&(e=c+i);var k=a.doc.height+Yb(f),l=g>c,m=e>k-g;if(h>c)j.scrollTop=l?0:c;else if(e>h+i){var n=Math.min(c,(m?k:e)-i);n!=h&&(j.scrollTop=n)}var o=a.curOp&&null!=a.curOp.scrollLeft?a.curOp.scrollLeft:f.scroller.scrollLeft,p=f.scroller.clientWidth-Bg-f.gutters.offsetWidth,q=d-b>p;return q&&(d=b+p),10>b?j.scrollLeft=0:o>b?j.scrollLeft=Math.max(0,b-(q?0:10)):d>p+o-3&&(j.scrollLeft=d+(q?0:10)-p),j}function ae(a,b,c){(null!=b||null!=c)&&ce(a),null!=b&&(a.curOp.scrollLeft=(null==a.curOp.scrollLeft?a.doc.scrollLeft:a.curOp.scrollLeft)+b),null!=c&&(a.curOp.scrollTop=(null==a.curOp.scrollTop?a.doc.scrollTop:a.curOp.scrollTop)+c)}function be(a){ce(a);var b=a.getCursor(),c=b,d=b;a.options.lineWrapping||(c=b.ch?mb(b.line,b.ch-1):b,d=mb(b.line,b.ch+1)),a.curOp.scrollToPos={from:c,to:d,margin:a.options.cursorScrollMargin,isCursor:!0}}function ce(a){var b=a.curOp.scrollToPos;if(b){a.curOp.scrollToPos=null;var c=rc(a,b.from),d=rc(a,b.to),e=_d(a,Math.min(c.left,d.left),Math.min(c.top,d.top)-b.margin,Math.max(c.right,d.right),Math.max(c.bottom,d.bottom)+b.margin);a.scrollTo(e.scrollLeft,e.scrollTop)}}function de(a,b,c,d){var f,e=a.doc;null==c&&(c="add"),"smart"==c&&(e.mode.indent?f=Wb(a,b):c="prev");var g=a.options.tabSize,h=Qf(e,b),i=Hg(h.text,null,g);h.stateAfter&&(h.stateAfter=null);var k,j=h.text.match(/^\s*/)[0];if(d||/\S/.test(h.text)){if("smart"==c&&(k=e.mode.indent(f,h.text.slice(j.length),h.text),k==Cg||k>150)){if(!d)return;c="prev"}}else k=0,c="not";"prev"==c?k=b>e.first?Hg(Qf(e,b-1).text,null,g):0:"add"==c?k=i+a.options.indentUnit:"subtract"==c?k=i-a.options.indentUnit:"number"==typeof c&&(k=i+c),k=Math.max(0,k);var l="",m=0;if(a.options.indentWithTabs)for(var n=Math.floor(k/g);n;--n)m+=g,l+=" ";if(k>m&&(l+=Kg(k-m)),l!=j)Xd(e,l,mb(b,0),mb(b,j.length),"+input");else for(var n=0;n=0;b--)Xd(a.doc,"",d[b].from,d[b].to,"+delete");be(a)})}function ge(a,b,c,d,e){function k(){var b=f+c;return b=a.first+a.size?j=!1:(f=b,i=Qf(a,b))}function l(a){var b=(e?Jh:Kh)(i,g,c,!0);if(null==b){if(a||!k())return j=!1;g=e?(0>c?Bh:Ah)(i):0>c?i.text.length:0}else g=b;return!0}var f=b.line,g=b.ch,h=c,i=Qf(a,f),j=!0;if("char"==d)l();else if("column"==d)l(!0);else if("word"==d||"group"==d)for(var m=null,n="group"==d,o=a.cm&&a.cm.getHelper(b,"wordChars"),p=!0;!(0>c)||l(!p);p=!1){var q=i.text.charAt(g)||"\n",r=Ug(q,o)?"w":n&&"\n"==q?"n":!n||/\s/.test(q)?null:"p";if(!n||p||r||(r="s"),m&&m!=r){0>c&&(c=1,l());break}if(r&&(m=r),c>0&&!l(!p))break}var s=Mb(a,mb(f,g),h,!0);return j||(s.hitSide=!0),s}function he(a,b,c,d){var g,e=a.doc,f=b.left;if("page"==d){var h=Math.min(a.display.wrapper.clientHeight,window.innerHeight||document.documentElement.clientHeight);g=b.top+c*(h-(0>c?1.5:.5)*wc(a.display))}else"line"==d&&(g=c>0?b.bottom+3:b.top-3);for(;;){var i=tc(a,f,g);if(!i.outside)break;if(0>c?0>=g:g>=e.height){i.hitSide=!0;break}g+=5*c}return i}function ke(a,b,c,d){w.defaults[a]=b,c&&(je[a]=d?function(a,b,d){d!=le&&c(a,b,d)}:c)}function ve(a){for(var c,d,e,f,b=a.split(/-(?!$)/),a=b[b.length-1],g=0;g0||0==g&&f.clearWhenEmpty!==!1)return f;if(f.replacedWith&&(f.collapsed=!0,f.widgetNode=Yg("span",[f.replacedWith],"CodeMirror-widget"),d.handleMouseEvents||(f.widgetNode.ignoreEvents=!0),d.insertLeft&&(f.widgetNode.insertLeft=!0)),f.collapsed){if(_e(a,b.line,b,c,f)||b.line!=c.line&&_e(a,c.line,b,c,f))throw new Error("Inserting collapsed marker partially overlapping an existing one"); v=!0}f.addToHistory&&ag(a,{from:b,to:c,origin:"markText"},a.sel,0/0);var j,h=b.line,i=a.cm;if(a.iter(h,c.line+1,function(a){i&&f.collapsed&&!i.options.lineWrapping&&af(a)==i.display.maxLine&&(j=!0),f.collapsed&&h!=b.line&&Tf(a,0),Me(a,new Je(f,h==b.line?b.ch:null,h==c.line?c.ch:null)),++h}),f.collapsed&&a.iter(b.line,c.line+1,function(b){ef(a,b)&&Tf(b,0)}),f.clearOnEnter&&rg(f,"beforeCursorEnter",function(){f.clear()}),f.readOnly&&(u=!0,(a.history.done.length||a.history.undone.length)&&a.clearHistory()),f.collapsed&&(f.id=++Ce,f.atomic=!0),i){if(j&&(i.curOp.updateMaxLine=!0),f.collapsed)Pc(i,b.line,c.line+1);else if(f.className||f.title||f.startStyle||f.endStyle)for(var k=b.line;k<=c.line;k++)Qc(i,k,"text");f.atomic&&Kb(i.doc),vg(i,"markerAdded",i,f)}return f}function Fe(a,b,c,d,e){d=Qg(d),d.shared=!1;var f=[De(a,b,c,d,e)],g=f[0],h=d.widgetNode;return Of(a,function(a){h&&(d.widgetNode=h.cloneNode(!0)),f.push(De(a,wb(a,b),wb(a,c),d,e));for(var i=0;i=b:f.to>b);(e||(e=[])).push(new Je(g,f.from,i?null:f.to))}}return e}function Oe(a,b,c){if(a)for(var e,d=0;d=b:f.to>b);if(h||f.from==b&&"bookmark"==g.type&&(!c||f.marker.insertLeft)){var i=null==f.from||(g.inclusiveLeft?f.from<=b:f.from0&&h)for(var l=0;ll;++l)o.push(q);o.push(i)}return o}function Qe(a){for(var b=0;b0)){var k=[i,1],l=nb(j.from,h.from),m=nb(j.to,h.to);(0>l||!g.inclusiveLeft&&!l)&&k.push({from:j.from,to:h.from}),(m>0||!g.inclusiveRight&&!m)&&k.push({from:h.to,to:j.to}),e.splice.apply(e,k),i+=k.length-1}}return e}function Te(a){var b=a.markedSpans;if(b){for(var c=0;c=0&&0>=l||0>=k&&l>=0)&&(0>=k&&(nb(j.to,c)>0||i.marker.inclusiveRight&&e.inclusiveLeft)||k>=0&&(nb(j.from,d)<0||i.marker.inclusiveLeft&&e.inclusiveRight)))return!0}}}function af(a){for(var b;b=Ze(a);)a=b.find(-1,!0).line;return a}function bf(a){for(var b,c;b=$e(a);)a=b.find(1,!0).line,(c||(c=[])).push(a);return c}function cf(a,b){var c=Qf(a,b),d=af(c);return c==d?b:Uf(d)}function df(a,b){if(b>a.lastLine())return b;var d,c=Qf(a,b);if(!ef(a,c))return b;for(;d=$e(c);)c=d.find(1,!0).line;return Uf(c)+1}function ef(a,b){var c=v&&b.markedSpans;if(c)for(var d,e=0;ee;e++){d&&(d[0]=w.innerMode(a,c).mode);var f=a.token(b,c);if(b.pos>b.start)return f}throw new Error("Mode "+a.name+" failed to advance stream.")}function rf(a,b,c,d){function e(a){return{start:k.start,end:k.pos,string:k.current(),type:h||null,state:a?re(f.mode,j):j}}var h,f=a.doc,g=f.mode;b=wb(f,b);var l,i=Qf(f,b.line),j=Wb(a,b.line,c),k=new Ae(i.text,a.options.tabSize);for(d&&(l=[]);(d||k.posa.options.maxHighlightLength?(h=!1,g&&vf(a,b,d,k.pos),k.pos=b.length,l=null):l=of(qf(c,k,d,m),f),m){var n=m[0].name;n&&(l="m-"+(l?n+" "+l:n))}h&&j==l||(ij;){var d=e[i];d>a&&e.splice(i,1,a,e[i+1],d),i+=2,j=Math.min(a,d)}if(b)if(h.opaque)e.splice(c,i-c,a,"cm-overlay "+b),i=c+2;else for(;i>c;c+=2){var f=e[c+1];e[c+1]=(f?f+" ":"")+"cm-overlay "+b}},f)}return{styles:e,classes:f.bgClass||f.textClass?f:null}}function uf(a,b,c){if(!b.styles||b.styles[0]!=a.state.modeGen){var d=tf(a,b,b.stateAfter=Wb(a,Uf(b)));b.styles=d.styles,d.classes?b.styleClasses=d.classes:b.styleClasses&&(b.styleClasses=null),c===a.doc.frontier&&a.doc.frontier++}return b.styles}function vf(a,b,c,d){var e=a.doc.mode,f=new Ae(b,a.options.tabSize);for(f.start=f.pos=d||0,""==b&&pf(e,c);!f.eol()&&f.pos<=a.options.maxHighlightLength;)qf(e,f,c),f.start=f.pos}function yf(a,b){if(!a||/^\s*$/.test(a))return null;var c=b.addModeClass?xf:wf;return c[a]||(c[a]=a.replace(/\S+/g,"cm-$&"))}function zf(a,b){var c=Yg("span",null,null,f?"padding-right: .1px":null),e={pre:Yg("pre",[c]),content:c,col:0,pos:0,cm:a};b.measure={};for(var g=0;g<=(b.rest?b.rest.length:0);g++){var i,h=g?b.rest[g-1]:b.line;e.pos=0,e.addToken=Bf,(d||f)&&a.getOption("lineWrapping")&&(e.addToken=Cf(e.addToken)),qh(a.display.measure)&&(i=Xf(h))&&(e.addToken=Df(e.addToken,i)),e.map=[];var j=b!=a.display.externalMeasured&&Uf(h);Ff(h,e,uf(a,h,j)),h.styleClasses&&(h.styleClasses.bgClass&&(e.bgClass=fh(h.styleClasses.bgClass,e.bgClass||"")),h.styleClasses.textClass&&(e.textClass=fh(h.styleClasses.textClass,e.textClass||""))),0==e.map.length&&e.map.push(0,0,e.content.appendChild(oh(a.display.measure))),0==g?(b.measure.map=e.map,b.measure.cache={}):((b.measure.maps||(b.measure.maps=[])).push(e.map),(b.measure.caches||(b.measure.caches=[])).push({}))}return f&&/\bcm-tab\b/.test(e.content.lastChild.className)&&(e.content.className="cm-tab-wrap-hack"),tg(a,"renderLine",a,b.line,e.pre),e.pre.className&&(e.textClass=fh(e.pre.className,e.textClass||"")),e}function Af(a){var b=Yg("span","\u2022","cm-invalidchar");return b.title="\\u"+a.charCodeAt(0).toString(16),b}function Bf(a,b,c,f,g,h){if(b){var i=a.cm.options.specialChars,j=!1;if(i.test(b))for(var k=document.createDocumentFragment(),l=0;;){i.lastIndex=l;var m=i.exec(b),n=m?m.index-l:b.length-l;if(n){var o=document.createTextNode(b.slice(l,l+n));d&&9>e?k.appendChild(Yg("span",[o])):k.appendChild(o),a.map.push(a.pos,a.pos+n,o),a.col+=n,a.pos+=n}if(!m)break;if(l+=n+1," "==m[0]){var p=a.cm.options.tabSize,q=p-a.col%p,o=k.appendChild(Yg("span",Kg(q),"cm-tab"));a.col+=q}else{var o=a.cm.options.specialCharPlaceholder(m[0]);d&&9>e?k.appendChild(Yg("span",[o])):k.appendChild(o),a.col+=1}a.map.push(a.pos,a.pos+1,o),a.pos++}else{a.col+=b.length;var k=document.createTextNode(b);a.map.push(a.pos,a.pos+b.length,k),d&&9>e&&(j=!0),a.pos+=b.length}if(c||f||g||j){var r=c||"";f&&(r+=f),g&&(r+=g);var s=Yg("span",[k],r);return h&&(s.title=h),a.content.appendChild(s)}a.content.appendChild(k)}}function Cf(a){function b(a){for(var b=" ",c=0;ci&&l.from<=i)break}if(l.to>=j)return a(c,d,e,f,g,h);a(c,d.slice(0,l.to-i),e,f,null,h),f=null,d=d.slice(l.to-i),i=l.to}}}function Ef(a,b,c,d){var e=!d&&c.widgetNode;e&&(a.map.push(a.pos,a.pos+b,e),a.content.appendChild(e)),a.pos+=b}function Ff(a,b,c){var d=a.markedSpans,e=a.text,f=0;if(d)for(var k,m,n,o,p,q,h=e.length,i=0,g=1,j="",l=0;;){if(l==i){m=n=o=p="",q=null,l=1/0;for(var r=[],s=0;si)?(null!=t.to&&l>t.to&&(l=t.to,n=""),u.className&&(m+=" "+u.className),u.startStyle&&t.from==i&&(o+=" "+u.startStyle),u.endStyle&&t.to==l&&(n+=" "+u.endStyle),u.title&&!p&&(p=u.title),u.collapsed&&(!q||Xe(q.marker,u)<0)&&(q=t)):t.from>i&&l>t.from&&(l=t.from),"bookmark"==u.type&&t.from==i&&u.widgetNode&&r.push(u)}if(q&&(q.from||0)==i&&(Ef(b,(null==q.to?h+1:q.to)-i,q.marker,null==q.from),null==q.to))return;if(!q&&r.length)for(var s=0;s=h)break;for(var v=Math.min(h,l);;){if(j){var w=i+j.length;if(!q){var x=w>v?j.slice(0,v-i):j;b.addToken(b,x,k?k+m:m,o,i+x.length==l?n:"",p)}if(w>=v){j=j.slice(v-i),i=v;break}i=w,o=""}j=e.slice(f,f=c[g++]),k=yf(c[g++],b.cm.options)}}else for(var g=1;g1&&a.remove(g.line+1,n-1),a.insert(g.line+1,p)}vg(a,"change",a,b)}function If(a){this.lines=a,this.parent=null;for(var b=0,c=0;bb||b>=a.size)throw new Error("There is no line "+(b+a.first)+" in the document.");for(var c=a;!c.lines;)for(var d=0;;++d){var e=c.children[d],f=e.chunkSize();if(f>b){c=e;break}b-=f}return c.lines[b]}function Rf(a,b,c){var d=[],e=b.line;return a.iter(b.line,c.line+1,function(a){var f=a.text;e==c.line&&(f=f.slice(0,c.ch)),e==b.line&&(f=f.slice(b.ch)),d.push(f),++e}),d}function Sf(a,b,c){var d=[];return a.iter(b,c,function(a){d.push(a.text)}),d}function Tf(a,b){var c=b-a.height;if(c)for(var d=a;d;d=d.parent)d.height+=c}function Uf(a){if(null==a.parent)return null;for(var b=a.parent,c=Ng(b.lines,a),d=b.parent;d;b=d,d=d.parent)for(var e=0;d.children[e]!=b;++e)c+=d.children[e].chunkSize();return c+b.first}function Vf(a,b){var c=a.first;a:do{for(var d=0;db){a=e;continue a}b-=f,c+=e.chunkSize()}return c}while(!a.lines);for(var d=0;db)break;b-=h}return c+d}function Wf(a){a=af(a);for(var b=0,c=a.parent,d=0;d1&&!a.done[a.done.length-2].ranges?(a.done.pop(),Lg(a.done)):void 0}function ag(a,b,c,d){var e=a.history;e.undone.length=0;var g,f=+new Date;if((e.lastOp==d||e.lastOrigin==b.origin&&b.origin&&("+"==b.origin.charAt(0)&&a.cm&&e.lastModTime>f-a.cm.options.historyEventDelay||"*"==b.origin.charAt(0)))&&(g=_f(e,e.lastOp==d))){var h=Lg(g.changes);0==nb(b.from,b.to)&&0==nb(b.from,h.to)?h.to=Ld(b):g.changes.push(Zf(a,b))}else{var i=Lg(e.done);for(i&&i.ranges||dg(a.sel,e.done),g={changes:[Zf(a,b)],generation:e.generation},e.done.push(g);e.done.length>e.undoDepth;)e.done.shift(),e.done[0].ranges||e.done.shift()}e.done.push(c),e.generation=++e.maxGeneration,e.lastModTime=e.lastSelTime=f,e.lastOp=e.lastSelOp=d,e.lastOrigin=e.lastSelOrigin=b.origin,h||tg(a,"historyAdded")}function bg(a,b,c,d){var e=b.charAt(0);return"*"==e||"+"==e&&c.ranges.length==d.ranges.length&&c.somethingSelected()==d.somethingSelected()&&new Date-a.history.lastSelTime<=(a.cm?a.cm.options.historyEventDelay:500)}function cg(a,b,c,d){var e=a.history,f=d&&d.origin;c==e.lastSelOp||f&&e.lastSelOrigin==f&&(e.lastModTime==e.lastSelTime&&e.lastOrigin==f||bg(a,f,Lg(e.done),b))?e.done[e.done.length-1]=b:dg(b,e.done),e.lastSelTime=+new Date,e.lastSelOrigin=f,e.lastSelOp=c,d&&d.clearRedo!==!1&&$f(e.undone)}function dg(a,b){var c=Lg(b);c&&c.ranges&&c.equals(a)||b.push(a)}function eg(a,b,c,d){var e=b["spans_"+a.id],f=0;a.iter(Math.max(a.first,c),Math.min(a.first+a.size,d),function(c){c.markedSpans&&((e||(e=b["spans_"+a.id]={}))[f]=c.markedSpans),++f})}function fg(a){if(!a)return null;for(var c,b=0;b-1&&(Lg(h)[l]=j[l],delete j[l])}}}return e}function ig(a,b,c,d){c0}function Ag(a){a.prototype.on=function(a,b){rg(this,a,b)},a.prototype.off=function(a,b){sg(this,a,b)}}function Gg(){this.id=null}function Ig(a,b,c){for(var d=0,e=0;;){var f=a.indexOf(" ",d);-1==f&&(f=a.length);var g=f-d;if(f==a.length||e+g>=b)return d+Math.min(g,b-e);if(e+=f-d,e+=c-e%c,d=f+1,e>=b)return d}}function Kg(a){for(;Jg.length<=a;)Jg.push(Lg(Jg)+" ");return Jg[a]}function Lg(a){return a[a.length-1]}function Ng(a,b){for(var c=0;c-1&&Tg(a)?!0:b.test(a):Tg(a)}function Vg(a){for(var b in a)if(a.hasOwnProperty(b)&&a[b])return!1;return!0}function Xg(a){return a.charCodeAt(0)>=768&&Wg.test(a)}function Yg(a,b,c,d){var e=document.createElement(a);if(c&&(e.className=c),d&&(e.style.cssText=d),"string"==typeof b)e.appendChild(document.createTextNode(b));else if(b)for(var f=0;f0;--b)a.removeChild(a.firstChild);return a}function _g(a,b){return $g(a).appendChild(b)}function ah(a,b){if(a.contains)return a.contains(b);for(;b=b.parentNode;)if(b==a)return!0}function bh(){return document.activeElement}function ch(a){return new RegExp("(^|\\s)"+a+"(?:$|\\s)\\s*")}function fh(a,b){for(var c=a.split(" "),d=0;d2&&!(d&&8>e))}return nh?Yg("span","\u200b"):Yg("span","\xa0",null,"display: inline-block; width: 1px; margin-right: -1px")}function qh(a){if(null!=ph)return ph;var b=_g(a,document.createTextNode("A\u062eA")),c=Zg(b,0,1).getBoundingClientRect();if(!c||c.left==c.right)return!1;var d=Zg(b,1,2).getBoundingClientRect();return ph=d.right-c.right<3}function vh(a){if(null!=uh)return uh;var b=_g(a,Yg("span","x")),c=b.getBoundingClientRect(),d=Zg(b,0,1).getBoundingClientRect();return uh=Math.abs(c.left-d.left)>1}function xh(a,b,c,d){if(!a)return d(b,c,"ltr");for(var e=!1,f=0;fb||b==c&&g.to==b)&&(d(Math.max(g.from,b),Math.min(g.to,c),1==g.level?"rtl":"ltr"),e=!0)}e||d(b,c,"ltr")}function yh(a){return a.level%2?a.to:a.from}function zh(a){return a.level%2?a.from:a.to}function Ah(a){var b=Xf(a);return b?yh(b[0]):0}function Bh(a){var b=Xf(a);return b?zh(Lg(b)):a.text.length}function Ch(a,b){var c=Qf(a.doc,b),d=af(c);d!=c&&(b=Uf(d));var e=Xf(d),f=e?e[0].level%2?Bh(d):Ah(d):0;return mb(b,f)}function Dh(a,b){for(var c,d=Qf(a.doc,b);c=$e(d);)d=c.find(1,!0).line,b=null;var e=Xf(d),f=e?e[0].level%2?Ah(d):Bh(d):d.text.length;return mb(null==b?Uf(d):b,f)}function Eh(a,b){var c=Ch(a,b.line),d=Qf(a.doc,c.line),e=Xf(d);if(!e||0==e[0].level){var f=Math.max(0,d.text.search(/\S/)),g=b.line==c.line&&b.ch<=f&&b.ch;return mb(c.line,g?0:f)}return c}function Fh(a,b,c){var d=a[0].level;return b==d?!0:c==d?!1:c>b}function Hh(a,b){Gh=null;for(var d,c=0;cb)return c;if(e.from==b||e.to==b){if(null!=d)return Fh(a,e.level,a[d].level)?(e.from!=e.to&&(Gh=d),c):(e.from!=e.to&&(Gh=c),d);d=c}}return d}function Ih(a,b,c,d){if(!d)return b+c;do b+=c;while(b>0&&Xg(a.text.charAt(b)));return b}function Jh(a,b,c,d){var e=Xf(a);if(!e)return Kh(a,b,c,d);for(var f=Hh(e,b),g=e[f],h=Ih(a,b,g.level%2?-c:c,d);;){if(h>g.from&&h0==g.level%2?g.to:g.from);if(g=e[f+=c],!g)return null;h=c>0==g.level%2?Ih(a,g.to,-1,d):Ih(a,g.from,1,d)}}function Kh(a,b,c,d){var e=b+c;if(d)for(;e>0&&Xg(a.text.charAt(e));)e+=c;return 0>e||e>a.text.length?null:e}var a=/gecko\/\d/i.test(navigator.userAgent),b=/MSIE \d/.test(navigator.userAgent),c=/Trident\/(?:[7-9]|\d{2,})\..*rv:(\d+)/.exec(navigator.userAgent),d=b||c,e=d&&(b?document.documentMode||6:c[1]),f=/WebKit\//.test(navigator.userAgent),g=f&&/Qt\/\d+\.\d+/.test(navigator.userAgent),h=/Chrome\//.test(navigator.userAgent),i=/Opera\//.test(navigator.userAgent),j=/Apple Computer/.test(navigator.vendor),k=/KHTML\//.test(navigator.userAgent),l=/Mac OS X 1\d\D([8-9]|\d\d)\D/.test(navigator.userAgent),m=/PhantomJS/.test(navigator.userAgent),n=/AppleWebKit/.test(navigator.userAgent)&&/Mobile\/\w+/.test(navigator.userAgent),o=n||/Android|webOS|BlackBerry|Opera Mini|Opera Mobi|IEMobile/i.test(navigator.userAgent),p=n||/Mac/.test(navigator.platform),q=/win/i.test(navigator.platform),r=i&&navigator.userAgent.match(/Version\/(\d*\.\d*)/);r&&(r=Number(r[1])),r&&r>=15&&(i=!1,f=!0);var s=p&&(g||i&&(null==r||12.11>r)),t=a||d&&e>=9,u=!1,v=!1,mb=w.Pos=function(a,b){return this instanceof mb?(this.line=a,this.ch=b,void 0):new mb(a,b)},nb=w.cmpPos=function(a,b){return a.line-b.line||a.ch-b.ch};rb.prototype={primary:function(){return this.ranges[this.primIndex]},equals:function(a){if(a==this)return!0;if(a.primIndex!=this.primIndex||a.ranges.length!=this.ranges.length)return!1;for(var b=0;b=0&&nb(a,d.to())<=0)return c}return-1}},sb.prototype={from:function(){return qb(this.anchor,this.head)},to:function(){return pb(this.anchor,this.head)},empty:function(){return this.head.line==this.anchor.line&&this.head.ch==this.anchor.ch}};var vc,hd,id,fc={left:0,right:0,top:0,bottom:0},yc=null,zc=0,Yc=null,od=0,td=0,ud=null;d?ud=-.53:a?ud=15:h?ud=-.7:j&&(ud=-1/3);var yd=new Gg,Cd=null,Ld=w.changeEnd=function(a){return a.text?mb(a.from.line+a.text.length-1,Lg(a.text).length+(1==a.text.length?a.from.ch:0)):a.to};w.prototype={constructor:w,focus:function(){window.focus(),_c(this),Xc(this)},setOption:function(a,b){var c=this.options,d=c[a];(c[a]!=b||"mode"==a)&&(c[a]=b,je.hasOwnProperty(a)&&Kc(this,je[a])(this,b,d))},getOption:function(a){return this.options[a]},getDoc:function(){return this.doc},addKeyMap:function(a,b){this.state.keyMaps[b?"push":"unshift"](ze(a))},removeKeyMap:function(a){for(var b=this.state.keyMaps,c=0;cc&&(de(this,e.head.line,a,!0),c=e.head.line,d==this.doc.sel.primIndex&&be(this));else{var f=e.from(),g=e.to(),h=Math.max(c,f.line);c=Math.min(this.lastLine(),g.line-(g.ch?0:1))+1;for(var i=h;c>i;++i)de(this,i,a);var j=this.doc.sel.ranges;0==f.ch&&b.length==j.length&&j[d].from().ch>0&&Db(this.doc,d,new sb(f,j[d].to()),Dg)}}}),getTokenAt:function(a,b){return rf(this,a,b)},getLineTokens:function(a,b){return rf(this,mb(a),b,!0)},getTokenTypeAt:function(a){a=wb(this.doc,a);var f,b=uf(this,Qf(this.doc,a.line)),c=0,d=(b.length-1)/2,e=a.ch;if(0==e)f=b[2];else for(;;){var g=c+d>>1;if((g?b[2*g-1]:0)>=e)d=g;else{if(!(b[2*g+1]h?f:0==h?null:f.slice(0,h-1)},getModeAt:function(a){var b=this.doc.mode;return b.innerMode?w.innerMode(b,this.getTokenAt(a).state).mode:b},getHelper:function(a,b){return this.getHelpers(a,b)[0]},getHelpers:function(a,b){var c=[];if(!qe.hasOwnProperty(b))return qe;var d=qe[b],e=this.getModeAt(a);if("string"==typeof e[b])d[e[b]]&&c.push(d[e[b]]);else if(e[b])for(var f=0;fd&&(a=d,c=!0);var e=Qf(this.doc,a);return nc(this,e,{top:0,left:0},b||"page").top+(c?this.doc.height-Wf(e):0)},defaultTextHeight:function(){return wc(this.display)},defaultCharWidth:function(){return xc(this.display)},setGutterMarker:Lc(function(a,b,c){return ee(this.doc,a,"gutter",function(a){var d=a.gutterMarkers||(a.gutterMarkers={});return d[b]=c,!c&&Vg(d)&&(a.gutterMarkers=null),!0})}),clearGutter:Lc(function(a){var b=this,c=b.doc,d=c.first;c.iter(function(c){c.gutterMarkers&&c.gutterMarkers[a]&&(c.gutterMarkers[a]=null,Qc(b,d,"gutter"),Vg(c.gutterMarkers)&&(c.gutterMarkers=null)),++d})}),addLineWidget:Lc(function(a,b,c){return kf(this,a,b,c)}),removeLineWidget:function(a){a.clear()},lineInfo:function(a){if("number"==typeof a){if(!yb(this.doc,a))return null;var b=a;if(a=Qf(this.doc,a),!a)return null}else{var b=Uf(a);if(null==b)return null}return{line:b,handle:a,text:a.text,gutterMarkers:a.gutterMarkers,textClass:a.textClass,bgClass:a.bgClass,wrapClass:a.wrapClass,widgets:a.widgets}},getViewport:function(){return{from:this.display.viewFrom,to:this.display.viewTo}},addWidget:function(a,b,c,d,e){var f=this.display;a=qc(this,wb(this.doc,a));var g=a.bottom,h=a.left;if(b.style.position="absolute",f.sizer.appendChild(b),"over"==d)g=a.top;else if("above"==d||"near"==d){var i=Math.max(f.wrapper.clientHeight,this.doc.height),j=Math.max(f.sizer.clientWidth,f.lineSpace.clientWidth); ("above"==d||a.bottom+b.offsetHeight>i)&&a.top>b.offsetHeight?g=a.top-b.offsetHeight:a.bottom+b.offsetHeight<=i&&(g=a.bottom),h+b.offsetWidth>j&&(h=j-b.offsetWidth)}b.style.top=g+"px",b.style.left=b.style.right="","right"==e?(h=f.sizer.clientWidth-b.offsetWidth,b.style.right="0px"):("left"==e?h=0:"middle"==e&&(h=(f.sizer.clientWidth-b.offsetWidth)/2),b.style.left=h+"px"),c&&$d(this,h,g,h+b.offsetWidth,g+b.offsetHeight)},triggerOnKeyDown:Lc(Dd),triggerOnKeyPress:Lc(Gd),triggerOnKeyUp:Fd,execCommand:function(a){return te.hasOwnProperty(a)?te[a](this):void 0},findPosH:function(a,b,c,d){var e=1;0>b&&(e=-1,b=-b);for(var f=0,g=wb(this.doc,a);b>f&&(g=ge(this.doc,g,e,c,d),!g.hitSide);++f);return g},moveH:Lc(function(a,b){var c=this;c.extendSelectionsBy(function(d){return c.display.shift||c.doc.extend||d.empty()?ge(c.doc,d.head,a,b,c.options.rtlMoveVisually):0>a?d.from():d.to()},Fg)}),deleteH:Lc(function(a,b){var c=this.doc.sel,d=this.doc;c.somethingSelected()?d.replaceSelection("",null,"+delete"):fe(this,function(c){var e=ge(d,c.head,a,b,!1);return 0>a?{from:e,to:c.head}:{from:c.head,to:e}})}),findPosV:function(a,b,c,d){var e=1,f=d;0>b&&(e=-1,b=-b);for(var g=0,h=wb(this.doc,a);b>g;++g){var i=qc(this,h,"div");if(null==f?f=i.left:i.left=f,h=he(this,i,e,c),h.hitSide)break}return h},moveV:Lc(function(a,b){var c=this,d=this.doc,e=[],f=!c.display.shift&&!d.extend&&d.sel.somethingSelected();if(d.extendSelectionsBy(function(g){if(f)return 0>a?g.from():g.to();var h=qc(c,g.head,"div");null!=g.goalColumn&&(h.left=g.goalColumn),e.push(h.left);var i=he(c,h,a,b);return"page"==b&&g==d.sel.primary()&&ae(c,null,pc(c,i,"div").top-h.top),i},Fg),e.length)for(var g=0;g0&&h(c.charAt(d-1));)--d;for(;e.5)&&C(this),tg(this,"refresh",this)}),swapDoc:Lc(function(a){var b=this.doc;return b.cm=null,Pf(this,a),kc(this),$c(this),this.scrollTo(a.scrollLeft,a.scrollTop),this.curOp.forceScroll=!0,vg(this,"swapDoc",this,b),b}),getInputField:function(){return this.display.input},getWrapperElement:function(){return this.display.wrapper},getScrollerElement:function(){return this.display.scroller},getGutterElement:function(){return this.display.gutters}},Ag(w);var ie=w.defaults={},je=w.optionHandlers={},le=w.Init={toString:function(){return"CodeMirror.Init"}};ke("value","",function(a,b){a.setValue(b)},!0),ke("mode",null,function(a,b){a.doc.modeOption=b,y(a)},!0),ke("indentUnit",2,y,!0),ke("indentWithTabs",!1),ke("smartIndent",!0),ke("tabSize",4,function(a){z(a),kc(a),Pc(a)},!0),ke("specialChars",/[\t\u0000-\u0019\u00ad\u200b-\u200f\u2028\u2029\ufeff]/g,function(a,b){a.options.specialChars=new RegExp(b.source+(b.test(" ")?"":"| "),"g"),a.refresh()},!0),ke("specialCharPlaceholder",Af,function(a){a.refresh()},!0),ke("electricChars",!0),ke("rtlMoveVisually",!q),ke("wholeLineUpdateBefore",!0),ke("theme","default",function(a){D(a),E(a)},!0),ke("keyMap","default",function(a,b,c){var d=ze(b),e=c!=w.Init&&ze(c);e&&e.detach&&e.detach(a,d),d.attach&&d.attach(a,e||null)}),ke("extraKeys",null),ke("lineWrapping",!1,A,!0),ke("gutters",[],function(a){J(a.options),E(a)},!0),ke("fixedGutter",!0,function(a,b){a.display.gutters.style.left=b?R(a.display)+"px":"0",a.refresh()},!0),ke("coverGutterNextToScrollbar",!1,M,!0),ke("lineNumbers",!1,function(a){J(a.options),E(a)},!0),ke("firstLineNumber",1,E,!0),ke("lineNumberFormatter",function(a){return a},E,!0),ke("showCursorWhenSelecting",!1,Pb,!0),ke("resetSelectionOnContextMenu",!0),ke("readOnly",!1,function(a,b){"nocursor"==b?(Id(a),a.display.input.blur(),a.display.disabled=!0):(a.display.disabled=!1,b||$c(a))}),ke("disableInput",!1,function(a,b){b||$c(a)},!0),ke("dragDrop",!0),ke("cursorBlinkRate",530),ke("cursorScrollMargin",0),ke("cursorHeight",1,Pb,!0),ke("singleCursorHeightPerLine",!0,Pb,!0),ke("workTime",100),ke("workDelay",100),ke("flattenSpans",!0,z,!0),ke("addModeClass",!1,z,!0),ke("pollInterval",100),ke("undoDepth",200,function(a,b){a.doc.history.undoDepth=b}),ke("historyEventDelay",1250),ke("viewportMargin",10,function(a){a.refresh()},!0),ke("maxHighlightLength",1e4,z,!0),ke("moveInputWithCursor",!0,function(a,b){b||(a.display.inputDiv.style.top=a.display.inputDiv.style.left=0)}),ke("tabindex",null,function(a,b){a.display.input.tabIndex=b||""}),ke("autofocus",null);var me=w.modes={},ne=w.mimeModes={};w.defineMode=function(a,b){w.defaults.mode||"null"==a||(w.defaults.mode=a),arguments.length>2&&(b.dependencies=Array.prototype.slice.call(arguments,2)),me[a]=b},w.defineMIME=function(a,b){ne[a]=b},w.resolveMode=function(a){if("string"==typeof a&&ne.hasOwnProperty(a))a=ne[a];else if(a&&"string"==typeof a.name&&ne.hasOwnProperty(a.name)){var b=ne[a.name];"string"==typeof b&&(b={name:b}),a=Pg(b,a),a.name=b.name}else if("string"==typeof a&&/^[\w\-]+\/[\w\-]+\+xml$/.test(a))return w.resolveMode("application/xml");return"string"==typeof a?{name:a}:a||{name:"null"}},w.getMode=function(a,b){var b=w.resolveMode(b),c=me[b.name];if(!c)return w.getMode(a,"text/plain");var d=c(a,b);if(oe.hasOwnProperty(b.name)){var e=oe[b.name];for(var f in e)e.hasOwnProperty(f)&&(d.hasOwnProperty(f)&&(d["_"+f]=d[f]),d[f]=e[f])}if(d.name=b.name,b.helperType&&(d.helperType=b.helperType),b.modeProps)for(var f in b.modeProps)d[f]=b.modeProps[f];return d},w.defineMode("null",function(){return{token:function(a){a.skipToEnd()}}}),w.defineMIME("text/plain","null");var oe=w.modeExtensions={};w.extendMode=function(a,b){var c=oe.hasOwnProperty(a)?oe[a]:oe[a]={};Qg(b,c)},w.defineExtension=function(a,b){w.prototype[a]=b},w.defineDocExtension=function(a,b){Lf.prototype[a]=b},w.defineOption=ke;var pe=[];w.defineInitHook=function(a){pe.push(a)};var qe=w.helpers={};w.registerHelper=function(a,b,c){qe.hasOwnProperty(a)||(qe[a]=w[a]={_global:[]}),qe[a][b]=c},w.registerGlobalHelper=function(a,b,c,d){w.registerHelper(a,b,d),qe[a]._global.push({pred:c,val:d})};var re=w.copyState=function(a,b){if(b===!0)return b;if(a.copyState)return a.copyState(b);var c={};for(var d in b){var e=b[d];e instanceof Array&&(e=e.concat([])),c[d]=e}return c},se=w.startState=function(a,b,c){return a.startState?a.startState(b,c):!0};w.innerMode=function(a,b){for(;a.innerMode;){var c=a.innerMode(b);if(!c||c.mode==a)break;b=c.state,a=c.mode}return c||{mode:a,state:b}};var te=w.commands={selectAll:function(a){a.setSelection(mb(a.firstLine(),0),mb(a.lastLine()),Dg)},singleSelection:function(a){a.setSelection(a.getCursor("anchor"),a.getCursor("head"),Dg)},killLine:function(a){fe(a,function(b){if(b.empty()){var c=Qf(a.doc,b.head.line).text.length;return b.head.ch==c&&b.head.line0)e=new mb(e.line,e.ch+1),a.replaceRange(f.charAt(e.ch-1)+f.charAt(e.ch-2),mb(e.line,e.ch-2),e,"+transpose");else if(e.line>a.doc.first){var g=Qf(a.doc,e.line-1).text;g&&a.replaceRange(f.charAt(0)+"\n"+g.charAt(g.length-1),mb(e.line-1,g.length-1),mb(e.line,1),"+transpose")}c.push(new sb(e,e))}a.setSelections(c)})},newlineAndIndent:function(a){Jc(a,function(){for(var b=a.listSelections().length,c=0;b>c;c++){var d=a.listSelections()[c];a.replaceRange("\n",d.anchor,d.head,"+input"),a.indentLine(d.from().line+1,null,!0),be(a)}})},toggleOverwrite:function(a){a.toggleOverwrite()}},ue=w.keyMap={};ue.basic={Left:"goCharLeft",Right:"goCharRight",Up:"goLineUp",Down:"goLineDown",End:"goLineEnd",Home:"goLineStartSmart",PageUp:"goPageUp",PageDown:"goPageDown",Delete:"delCharAfter",Backspace:"delCharBefore","Shift-Backspace":"delCharBefore",Tab:"defaultTab","Shift-Tab":"indentAuto",Enter:"newlineAndIndent",Insert:"toggleOverwrite",Esc:"singleSelection"},ue.pcDefault={"Ctrl-A":"selectAll","Ctrl-D":"deleteLine","Ctrl-Z":"undo","Shift-Ctrl-Z":"redo","Ctrl-Y":"redo","Ctrl-Home":"goDocStart","Ctrl-End":"goDocEnd","Ctrl-Up":"goLineUp","Ctrl-Down":"goLineDown","Ctrl-Left":"goGroupLeft","Ctrl-Right":"goGroupRight","Alt-Left":"goLineStart","Alt-Right":"goLineEnd","Ctrl-Backspace":"delGroupBefore","Ctrl-Delete":"delGroupAfter","Ctrl-S":"save","Ctrl-F":"find","Ctrl-G":"findNext","Shift-Ctrl-G":"findPrev","Shift-Ctrl-F":"replace","Shift-Ctrl-R":"replaceAll","Ctrl-[":"indentLess","Ctrl-]":"indentMore","Ctrl-U":"undoSelection","Shift-Ctrl-U":"redoSelection","Alt-U":"redoSelection",fallthrough:"basic"},ue.emacsy={"Ctrl-F":"goCharRight","Ctrl-B":"goCharLeft","Ctrl-P":"goLineUp","Ctrl-N":"goLineDown","Alt-F":"goWordRight","Alt-B":"goWordLeft","Ctrl-A":"goLineStart","Ctrl-E":"goLineEnd","Ctrl-V":"goPageDown","Shift-Ctrl-V":"goPageUp","Ctrl-D":"delCharAfter","Ctrl-H":"delCharBefore","Alt-D":"delWordAfter","Alt-Backspace":"delWordBefore","Ctrl-K":"killLine","Ctrl-T":"transposeChars"},ue.macDefault={"Cmd-A":"selectAll","Cmd-D":"deleteLine","Cmd-Z":"undo","Shift-Cmd-Z":"redo","Cmd-Y":"redo","Cmd-Home":"goDocStart","Cmd-Up":"goDocStart","Cmd-End":"goDocEnd","Cmd-Down":"goDocEnd","Alt-Left":"goGroupLeft","Alt-Right":"goGroupRight","Cmd-Left":"goLineLeft","Cmd-Right":"goLineRight","Alt-Backspace":"delGroupBefore","Ctrl-Alt-Backspace":"delGroupAfter","Alt-Delete":"delGroupAfter","Cmd-S":"save","Cmd-F":"find","Cmd-G":"findNext","Shift-Cmd-G":"findPrev","Cmd-Alt-F":"replace","Shift-Cmd-Alt-F":"replaceAll","Cmd-[":"indentLess","Cmd-]":"indentMore","Cmd-Backspace":"delWrappedLineLeft","Cmd-Delete":"delWrappedLineRight","Cmd-U":"undoSelection","Shift-Cmd-U":"redoSelection","Ctrl-Up":"goDocStart","Ctrl-Down":"goDocEnd",fallthrough:["basic","emacsy"]},ue["default"]=p?ue.macDefault:ue.pcDefault,w.normalizeKeyMap=function(a){var b={};for(var c in a)if(a.hasOwnProperty(c)){var d=a[c];if(/^(name|fallthrough|(de|at)tach)$/.test(c))continue;if("..."==d){delete a[c];continue}for(var e=Og(c.split(" "),ve),f=0;f=this.string.length},sol:function(){return this.pos==this.lineStart},peek:function(){return this.string.charAt(this.pos)||void 0},next:function(){return this.posb},eatSpace:function(){for(var a=this.pos;/[\s\u00a0]/.test(this.string.charAt(this.pos));)++this.pos;return this.pos>a},skipToEnd:function(){this.pos=this.string.length},skipTo:function(a){var b=this.string.indexOf(a,this.pos);return b>-1?(this.pos=b,!0):void 0},backUp:function(a){this.pos-=a},column:function(){return this.lastColumnPos0?null:(f&&b!==!1&&(this.pos+=f[0].length),f)}var d=function(a){return c?a.toLowerCase():a},e=this.string.substr(this.pos,a.length);return d(e)==d(a)?(b!==!1&&(this.pos+=a.length),!0):void 0},current:function(){return this.string.slice(this.start,this.pos)},hideFirstChars:function(a,b){this.lineStart+=a;try{return b()}finally{this.lineStart-=a}}};var Be=w.TextMarker=function(a,b){this.lines=[],this.type=b,this.doc=a};Ag(Be),Be.prototype.clear=function(){if(!this.explicitlyCleared){var a=this.doc.cm,b=a&&!a.curOp;if(b&&Ac(a),zg(this,"clear")){var c=this.find();c&&vg(this,"clear",c.from,c.to)}for(var d=null,e=null,f=0;fa.display.maxLineLength&&(a.display.maxLine=i,a.display.maxLineLength=j,a.display.maxLineChanged=!0)}null!=d&&a&&this.collapsed&&Pc(a,d,e+1),this.lines.length=0,this.explicitlyCleared=!0,this.atomic&&this.doc.cantEdit&&(this.doc.cantEdit=!1,a&&Kb(a.doc)),a&&vg(a,"markerCleared",a,this),b&&Cc(a),this.parent&&this.parent.clear()}},Be.prototype.find=function(a,b){null==a&&"bookmark"==this.type&&(a=1);for(var c,d,e=0;ec;++c){var e=this.lines[c];this.height-=e.height,nf(e),vg(e,"delete")}this.lines.splice(a,b)},collapse:function(a){a.push.apply(a,this.lines)},insertInner:function(a,b,c){this.height+=c,this.lines=this.lines.slice(0,a).concat(b).concat(this.lines.slice(a));for(var d=0;da;++a)if(c(this.lines[a]))return!0}},Jf.prototype={chunkSize:function(){return this.size},removeInner:function(a,b){this.size-=b;for(var c=0;ca){var f=Math.min(b,e-a),g=d.height;if(d.removeInner(a,f),this.height-=g-d.height,e==f&&(this.children.splice(c--,1),d.parent=null),0==(b-=f))break;a=0}else a-=e}if(this.size-b<25&&(this.children.length>1||!(this.children[0]instanceof If))){var h=[];this.collapse(h),this.children=[new If(h)],this.children[0].parent=this}},collapse:function(a){for(var b=0;b=a){if(e.insertInner(a,b,c),e.lines&&e.lines.length>50){for(;e.lines.length>50;){var g=e.lines.splice(e.lines.length-25,25),h=new If(g);e.height-=h.height,this.children.splice(d+1,0,h),h.parent=this}this.maybeSpill()}break}a-=f}},maybeSpill:function(){if(!(this.children.length<=10)){var a=this;do{var b=a.children.splice(a.children.length-5,5),c=new Jf(b);if(a.parent){a.size-=c.size,a.height-=c.height;var e=Ng(a.parent.children,a);a.parent.children.splice(e+1,0,c)}else{var d=new Jf(a.children);d.parent=a,a.children=[d,c],a=d}c.parent=a.parent}while(a.children.length>10);a.parent.maybeSpill()}},iterN:function(a,b,c){for(var d=0;da){var g=Math.min(b,f-a);if(e.iterN(a,g,c))return!0;if(0==(b-=g))break;a=0}else a-=f}}};var Kf=0,Lf=w.Doc=function(a,b,c){if(!(this instanceof Lf))return new Lf(a,b,c);null==c&&(c=0),Jf.call(this,[new If([new lf("",null)])]),this.first=c,this.scrollTop=this.scrollLeft=0,this.cantEdit=!1,this.cleanGeneration=1,this.frontier=c;var d=mb(c,0);this.sel=ub(d),this.history=new Yf(null),this.id=++Kf,this.modeOption=b,"string"==typeof a&&(a=rh(a)),Hf(this,{from:d,to:d,text:a}),Hb(this,ub(d),Dg)};Lf.prototype=Pg(Jf.prototype,{constructor:Lf,iter:function(a,b,c){c?this.iterN(a-this.first,b-a,c):this.iterN(this.first,this.first+this.size,a)},insert:function(a,b){for(var c=0,d=0;d=0;f--)Rd(this,d[f]);h?Gb(this,h):this.cm&&be(this.cm)}),undo:Mc(function(){Td(this,"undo")}),redo:Mc(function(){Td(this,"redo")}),undoSelection:Mc(function(){Td(this,"undo",!0)}),redoSelection:Mc(function(){Td(this,"redo",!0)}),setExtending:function(a){this.extend=a},getExtending:function(){return this.extend},historySize:function(){for(var a=this.history,b=0,c=0,d=0;d=a.ch)&&b.push(e.marker.parent||e.marker)}return b},findMarks:function(a,b,c){a=wb(this,a),b=wb(this,b);var d=[],e=a.line;return this.iter(a.line,b.line+1,function(f){var g=f.markedSpans;if(g)for(var h=0;hi.to||null==i.from&&e!=a.line||e==b.line&&i.from>b.ch||c&&!c(i.marker)||d.push(i.marker.parent||i.marker)}++e}),d},getAllMarks:function(){var a=[];return this.iter(function(b){var c=b.markedSpans;if(c)for(var d=0;da?(b=a,!0):(a-=e,++c,void 0)}),wb(this,mb(c,b))},indexFromPos:function(a){a=wb(this,a);var b=a.ch;return a.lineb&&(b=a.from),null!=a.to&&a.toh||h>=b)return g+(b-f);g+=h-f,g+=c-g%c,f=h+1}},Jg=[""],Mg=function(a){a.select()};n?Mg=function(a){a.selectionStart=0,a.selectionEnd=a.value.length}:d&&(Mg=function(a){try{a.select()}catch(b){}}),[].indexOf&&(Ng=function(a,b){return a.indexOf(b)}),[].map&&(Og=function(a,b){return a.map(b)});var Zg,Sg=/[\u00df\u0590-\u05f4\u0600-\u06ff\u3040-\u309f\u30a0-\u30ff\u3400-\u4db5\u4e00-\u9fcc\uac00-\ud7af]/,Tg=w.isWordChar=function(a){return/\w/.test(a)||a>"\x80"&&(a.toUpperCase()!=a.toLowerCase()||Sg.test(a))},Wg=/[\u0300-\u036f\u0483-\u0489\u0591-\u05bd\u05bf\u05c1\u05c2\u05c4\u05c5\u05c7\u0610-\u061a\u064b-\u065e\u0670\u06d6-\u06dc\u06de-\u06e4\u06e7\u06e8\u06ea-\u06ed\u0711\u0730-\u074a\u07a6-\u07b0\u07eb-\u07f3\u0816-\u0819\u081b-\u0823\u0825-\u0827\u0829-\u082d\u0900-\u0902\u093c\u0941-\u0948\u094d\u0951-\u0955\u0962\u0963\u0981\u09bc\u09be\u09c1-\u09c4\u09cd\u09d7\u09e2\u09e3\u0a01\u0a02\u0a3c\u0a41\u0a42\u0a47\u0a48\u0a4b-\u0a4d\u0a51\u0a70\u0a71\u0a75\u0a81\u0a82\u0abc\u0ac1-\u0ac5\u0ac7\u0ac8\u0acd\u0ae2\u0ae3\u0b01\u0b3c\u0b3e\u0b3f\u0b41-\u0b44\u0b4d\u0b56\u0b57\u0b62\u0b63\u0b82\u0bbe\u0bc0\u0bcd\u0bd7\u0c3e-\u0c40\u0c46-\u0c48\u0c4a-\u0c4d\u0c55\u0c56\u0c62\u0c63\u0cbc\u0cbf\u0cc2\u0cc6\u0ccc\u0ccd\u0cd5\u0cd6\u0ce2\u0ce3\u0d3e\u0d41-\u0d44\u0d4d\u0d57\u0d62\u0d63\u0dca\u0dcf\u0dd2-\u0dd4\u0dd6\u0ddf\u0e31\u0e34-\u0e3a\u0e47-\u0e4e\u0eb1\u0eb4-\u0eb9\u0ebb\u0ebc\u0ec8-\u0ecd\u0f18\u0f19\u0f35\u0f37\u0f39\u0f71-\u0f7e\u0f80-\u0f84\u0f86\u0f87\u0f90-\u0f97\u0f99-\u0fbc\u0fc6\u102d-\u1030\u1032-\u1037\u1039\u103a\u103d\u103e\u1058\u1059\u105e-\u1060\u1071-\u1074\u1082\u1085\u1086\u108d\u109d\u135f\u1712-\u1714\u1732-\u1734\u1752\u1753\u1772\u1773\u17b7-\u17bd\u17c6\u17c9-\u17d3\u17dd\u180b-\u180d\u18a9\u1920-\u1922\u1927\u1928\u1932\u1939-\u193b\u1a17\u1a18\u1a56\u1a58-\u1a5e\u1a60\u1a62\u1a65-\u1a6c\u1a73-\u1a7c\u1a7f\u1b00-\u1b03\u1b34\u1b36-\u1b3a\u1b3c\u1b42\u1b6b-\u1b73\u1b80\u1b81\u1ba2-\u1ba5\u1ba8\u1ba9\u1c2c-\u1c33\u1c36\u1c37\u1cd0-\u1cd2\u1cd4-\u1ce0\u1ce2-\u1ce8\u1ced\u1dc0-\u1de6\u1dfd-\u1dff\u200c\u200d\u20d0-\u20f0\u2cef-\u2cf1\u2de0-\u2dff\u302a-\u302f\u3099\u309a\ua66f-\ua672\ua67c\ua67d\ua6f0\ua6f1\ua802\ua806\ua80b\ua825\ua826\ua8c4\ua8e0-\ua8f1\ua926-\ua92d\ua947-\ua951\ua980-\ua982\ua9b3\ua9b6-\ua9b9\ua9bc\uaa29-\uaa2e\uaa31\uaa32\uaa35\uaa36\uaa43\uaa4c\uaab0\uaab2-\uaab4\uaab7\uaab8\uaabe\uaabf\uaac1\uabe5\uabe8\uabed\udc00-\udfff\ufb1e\ufe00-\ufe0f\ufe20-\ufe26\uff9e\uff9f]/;Zg=document.createRange?function(a,b,c){var d=document.createRange();return d.setEnd(a,c),d.setStart(a,b),d}:function(a,b,c){var d=document.body.createTextRange();try{d.moveToElementText(a.parentNode)}catch(e){return d}return d.collapse(!0),d.moveEnd("character",c),d.moveStart("character",b),d},d&&11>e&&(bh=function(){try{return document.activeElement}catch(a){return document.body}});var lh,nh,ph,dh=w.rmClass=function(a,b){var c=a.className,d=ch(b).exec(c);if(d){var e=c.slice(d.index+d[0].length);a.className=c.slice(0,d.index)+(e?d[1]+e:"")}},eh=w.addClass=function(a,b){var c=a.className;ch(b).test(c)||(a.className+=(c?" ":"")+b)},hh=!1,kh=function(){if(d&&9>e)return!1;var a=Yg("div");return"draggable"in a||"dragDrop"in a}(),rh=w.splitLines=3!="\n\nb".split(/\n/).length?function(a){for(var b=0,c=[],d=a.length;d>=b;){var e=a.indexOf("\n",b);-1==e&&(e=a.length);var f=a.slice(b,"\r"==a.charAt(e-1)?e-1:e),g=f.indexOf("\r");-1!=g?(c.push(f.slice(0,g)),b+=g+1):(c.push(f),b=e+1)}return c}:function(a){return a.split(/\r\n?|\n/)},sh=window.getSelection?function(a){try{return a.selectionStart!=a.selectionEnd}catch(b){return!1}}:function(a){try{var b=a.ownerDocument.selection.createRange()}catch(c){}return b&&b.parentElement()==a?0!=b.compareEndPoints("StartToEnd",b):!1},th=function(){var a=Yg("div");return"oncopy"in a?!0:(a.setAttribute("oncopy","return;"),"function"==typeof a.oncopy)}(),uh=null,wh={3:"Enter",8:"Backspace",9:"Tab",13:"Enter",16:"Shift",17:"Ctrl",18:"Alt",19:"Pause",20:"CapsLock",27:"Esc",32:"Space",33:"PageUp",34:"PageDown",35:"End",36:"Home",37:"Left",38:"Up",39:"Right",40:"Down",44:"PrintScrn",45:"Insert",46:"Delete",59:";",61:"=",91:"Mod",92:"Mod",93:"Mod",107:"=",109:"-",127:"Delete",173:"-",186:";",187:"=",188:",",189:"-",190:".",191:"/",192:"`",219:"[",220:"\\",221:"]",222:"'",63232:"Up",63233:"Down",63234:"Left",63235:"Right",63272:"Delete",63273:"Home",63275:"End",63276:"PageUp",63277:"PageDown",63302:"Insert"};w.keyNames=wh,function(){for(var a=0;10>a;a++)wh[a+48]=wh[a+96]=String(a);for(var a=65;90>=a;a++)wh[a]=String.fromCharCode(a);for(var a=1;12>=a;a++)wh[a+111]=wh[a+63235]="F"+a}();var Gh,Lh=function(){function c(c){return 247>=c?a.charAt(c):c>=1424&&1524>=c?"R":c>=1536&&1773>=c?b.charAt(c-1536):c>=1774&&2220>=c?"r":c>=8192&&8203>=c?"w":8204==c?"b":"L"}function j(a,b,c){this.level=a,this.from=b,this.to=c}var a="bbbbbbbbbtstwsbbbbbbbbbbbbbbssstwNN%%%NNNNNN,N,N1111111111NNNNNNNLLLLLLLLLLLLLLLLLLLLLLLLLLNNNNNNLLLLLLLLLLLLLLLLLLLLLLLLLLNNNNbbbbbbsbbbbbbbbbbbbbbbbbbbbbbbbbb,N%%%%NNNNLNNNNN%%11NLNNN1LNNNNNLLLLLLLLLLLLLLLLLLLLLLLNLLLLLLLLLLLLLLLLLLLLLLLLLLLLLLLN",b="rrrrrrrrrrrr,rNNmmmmmmrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrmmmmmmmmmmmmmmrrrrrrrnnnnnnnnnn%nnrrrmrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrmmmmmmmmmmmmmmmmmmmNmmmm",d=/[\u0590-\u05f4\u0600-\u06ff\u0700-\u08ac]/,e=/[stwN]/,f=/[LRr]/,g=/[Lb1n]/,h=/[1n]/,i="L";return function(a){if(!d.test(a))return!1;for(var m,b=a.length,k=[],l=0;b>l;++l)k.push(m=c(a.charCodeAt(l)));for(var l=0,n=i;b>l;++l){var m=k[l];"m"==m?k[l]=n:n=m}for(var l=0,o=i;b>l;++l){var m=k[l];"1"==m&&"r"==o?k[l]="n":f.test(m)&&(o=m,"r"==m&&(k[l]="R"))}for(var l=1,n=k[0];b-1>l;++l){var m=k[l];"+"==m&&"1"==n&&"1"==k[l+1]?k[l]="1":","!=m||n!=k[l+1]||"1"!=n&&"n"!=n||(k[l]=n),n=m}for(var l=0;b>l;++l){var m=k[l];if(","==m)k[l]="N";else if("%"==m){for(var p=l+1;b>p&&"%"==k[p];++p);for(var q=l&&"!"==k[l-1]||b>p&&"1"==k[p]?"1":"N",r=l;p>r;++r)k[r]=q;l=p-1}}for(var l=0,o=i;b>l;++l){var m=k[l];"L"==o&&"1"==m?k[l]="L":f.test(m)&&(o=m)}for(var l=0;b>l;++l)if(e.test(k[l])){for(var p=l+1;b>p&&e.test(k[p]);++p);for(var s="L"==(l?k[l-1]:i),t="L"==(b>p?k[p]:i),q=s||t?"L":"R",r=l;p>r;++r)k[r]=q;l=p-1}for(var v,u=[],l=0;b>l;)if(g.test(k[l])){var w=l;for(++l;b>l&&g.test(k[l]);++l);u.push(new j(0,w,l))}else{var x=l,y=u.length;for(++l;b>l&&"L"!=k[l];++l);for(var r=x;l>r;)if(h.test(k[r])){r>x&&u.splice(y,0,new j(1,x,r));var z=r;for(++r;l>r&&h.test(k[r]);++r);u.splice(y,0,new j(2,z,r)),x=r}else++r;l>x&&u.splice(y,0,new j(1,x,l))}return 1==u[0].level&&(v=a.match(/^\s+/))&&(u[0].from=v[0].length,u.unshift(new j(0,0,v[0].length))),1==Lg(u).level&&(v=a.match(/\s+$/))&&(Lg(u).to-=v[0].length,u.push(new j(0,b-v[0].length,b))),u[0].level!=Lg(u).level&&u.push(new j(u[0].level,b,b)),u}}();return w.version="4.8.0",w}),function(a){"object"==typeof exports&&"object"==typeof module?a(require("../../lib/codemirror")):"function"==typeof define&&define.amd?define(["../../lib/codemirror"],a):a(CodeMirror)}(function(a){function e(a,b,e,g){var h=a.getLineHandle(b.line),i=b.ch-1,j=i>=0&&d[h.text.charAt(i)]||d[h.text.charAt(++i)];if(!j)return null;var k=">"==j.charAt(1)?1:-1;if(e&&k>0!=(i==b.ch))return null;var l=a.getTokenTypeAt(c(b.line,i+1)),m=f(a,c(b.line,i+(k>0?1:0)),k,l||null,g);return null==m?null:{from:c(b.line,i),to:m&&m.pos,match:m&&m.ch==j.charAt(0),forward:k>0}}function f(a,b,e,f,g){for(var h=g&&g.maxScanLineLength||1e4,i=g&&g.maxScanLines||1e3,j=[],k=g&&g.bracketRegex?g.bracketRegex:/[(){}[\]]/,l=e>0?Math.min(b.line+i,a.lastLine()+1):Math.max(a.firstLine()-1,b.line-i),m=b.line;m!=l;m+=e){var n=a.getLine(m);if(n){var o=e>0?0:n.length-1,p=e>0?n.length:-1;if(!(n.length>h))for(m==b.line&&(o=b.ch-(0>e?1:0));o!=p;o+=e){var q=n.charAt(o);if(k.test(q)&&(void 0===f||a.getTokenTypeAt(c(m,o+1))==f)){var r=d[q];if(">"==r.charAt(1)==e>0)j.push(q);else{if(!j.length)return{pos:c(m,o),ch:q};j.pop()}}}}}return m-e==(e>0?a.lastLine():a.firstLine())?!1:null}function g(a,d,f){for(var g=a.state.matchBrackets.maxHighlightLineLength||1e3,h=[],i=a.listSelections(),j=0;j",")":"(<","[":"]>","]":"[<","{":"}>","}":"{<"},h=null;a.defineOption("matchBrackets",!1,function(b,c,d){d&&d!=a.Init&&b.off("cursorActivity",i),c&&(b.state.matchBrackets="object"==typeof c?c:{},b.on("cursorActivity",i))}),a.defineExtension("matchBrackets",function(){g(this,!0)}),a.defineExtension("findMatchingBracket",function(a,b,c){return e(this,a,b,c)}),a.defineExtension("scanForBracket",function(a,b,c,d){return f(this,a,b,c,d)})}),function(a){"object"==typeof exports&&"object"==typeof module?a(require("../../lib/codemirror")):"function"==typeof define&&define.amd?define(["../../lib/codemirror"],a):a(CodeMirror)}(function(a){"use strict";function d(a,b){this.cm=a,this.options=this.buildOptions(b),this.widget=this.onClose=null}function e(a){return"string"==typeof a?a:a.text}function f(a,b){function f(a,d){var f;f="string"!=typeof d?function(a){return d(a,b)}:c.hasOwnProperty(d)?c[d]:d,e[a]=f}var c={Up:function(){b.moveFocus(-1)},Down:function(){b.moveFocus(1)},PageUp:function(){b.moveFocus(-b.menuSize()+1,!0)},PageDown:function(){b.moveFocus(b.menuSize()-1,!0)},Home:function(){b.setFocus(0)},End:function(){b.setFocus(b.length-1)},Enter:b.pick,Tab:b.pick,Esc:b.close},d=a.options.customKeys,e=d?{}:c;if(d)for(var g in d)d.hasOwnProperty(g)&&f(g,d[g]);var h=a.options.extraKeys;if(h)for(var g in h)h.hasOwnProperty(g)&&f(g,h[g]);return e}function g(a,b){for(;b&&b!=a;){if("LI"===b.nodeName.toUpperCase()&&b.parentNode==a)return b;b=b.parentNode}}function h(d,h){this.completion=d,this.data=h;var i=this,j=d.cm,k=this.hints=document.createElement("ul");k.className="CodeMirror-hints",this.selectedHint=h.selectedHint||0;for(var l=h.list,m=0;m0){var y=w.bottom-w.top,z=q.top-(q.bottom-w.top);if(z-y>0)k.style.top=(s=q.top-y)+"px",t=!1;else if(y>v){k.style.height=v-5+"px",k.style.top=(s=q.bottom-w.top)+"px";var A=j.getCursor();h.from.ch!=A.ch&&(q=j.cursorCoords(A),k.style.left=(r=q.left)+"px",w=k.getBoundingClientRect())}}var B=w.left-u;if(B>0&&(w.right-w.left>u&&(k.style.width=u-5+"px",B-=w.right-w.left-u),k.style.left=(r=q.left-B)+"px"),j.addKeyMap(this.keyMap=f(d,{moveFocus:function(a,b){i.changeActive(i.selectedHint+a,b)},setFocus:function(a){i.changeActive(a)},menuSize:function(){return i.screenAmount()},length:l.length,close:function(){d.close()},pick:function(){i.pick()},data:h})),d.options.closeOnUnfocus){var C;j.on("blur",this.onBlur=function(){C=setTimeout(function(){d.close()},100)}),j.on("focus",this.onFocus=function(){clearTimeout(C)})}var D=j.getScrollInfo();return j.on("scroll",this.onScroll=function(){var a=j.getScrollInfo(),b=j.getWrapperElement().getBoundingClientRect(),c=s+D.top-a.top,e=c-(window.pageYOffset||(document.documentElement||document.body).scrollTop);return t||(e+=k.offsetHeight),e<=b.top||e>=b.bottom?d.close():(k.style.top=c+"px",k.style.left=r+D.left-a.left+"px",void 0)}),a.on(k,"dblclick",function(a){var b=g(k,a.target||a.srcElement);b&&null!=b.hintId&&(i.changeActive(b.hintId),i.pick())}),a.on(k,"click",function(a){var b=g(k,a.target||a.srcElement);b&&null!=b.hintId&&(i.changeActive(b.hintId),d.options.completeOnSingleClick&&i.pick())}),a.on(k,"mousedown",function(){setTimeout(function(){j.focus()},20)}),a.signal(h,"select",l[0],k.firstChild),!0}var b="CodeMirror-hint",c="CodeMirror-hint-active";a.showHint=function(a,b,c){if(!b)return a.showHint(c);c&&c.async&&(b.async=!0);var d={hint:b};if(c)for(var e in c)d[e]=c[e];return a.showHint(d)},a.defineExtension("showHint",function(b){if(!(this.listSelections().length>1||this.somethingSelected())){this.state.completionActive&&this.state.completionActive.close();var c=this.state.completionActive=new d(this,b),e=c.options.hint;if(e)return a.signal(this,"startCompletion",this),e.async?(e(this,function(a){c.showHints(a)},c.options),void 0):c.showHints(e(this,c.options))}}),d.prototype={close:function(){this.active()&&(this.cm.state.completionActive=null,this.widget&&this.widget.close(),this.onClose&&this.onClose(),a.signal(this.cm,"endCompletion",this.cm))},active:function(){return this.cm.state.completionActive==this},pick:function(b,c){var d=b.list[c];d.hint?d.hint(this.cm,b,d):this.cm.replaceRange(e(d),d.from||b.from,d.to||b.to,"complete"),a.signal(b,"pick",d),this.close()},showHints:function(a){return a&&a.list.length&&this.active()?(this.options.completeSingle&&1==a.list.length?this.pick(a,0):this.showWidget(a),void 0):this.close()},showWidget:function(b){function l(){e||(e=!0,d.close(),d.cm.off("cursorActivity",p),b&&a.signal(b,"close"))}function m(){if(!e){a.signal(b,"update");var c=d.options.hint;c.async?c(d.cm,n,d.options):n(c(d.cm,d.options))}}function n(a){if(b=a,!e){if(!b||!b.list.length)return l();d.widget&&d.widget.close(),d.widget=new h(d,b)}}function o(){c&&(k(c),c=0)}function p(){o();var a=d.cm.getCursor(),b=d.cm.getLine(a.line);a.line!=g.line||b.length-a.ch!=i-g.ch||a.ch=this.data.list.length?b=d?this.data.list.length-1:0:0>b&&(b=d?0:this.data.list.length-1),this.selectedHint!=b){var e=this.hints.childNodes[this.selectedHint];e.className=e.className.replace(" "+c,""),e=this.hints.childNodes[this.selectedHint=b],e.className+=" "+c,e.offsetTopthis.hints.scrollTop+this.hints.clientHeight&&(this.hints.scrollTop=e.offsetTop+e.offsetHeight-this.hints.clientHeight+3),a.signal(this.data,"select",this.data.list[this.selectedHint],e)}},screenAmount:function(){return Math.floor(this.hints.clientHeight/this.hints.firstChild.offsetHeight)||1}},a.registerHelper("hint","auto",function(b,c){var e,d=b.getHelpers(b.getCursor(),"hint");if(d.length)for(var f=0;f,]/,closeOnUnfocus:!0,completeOnSingleClick:!1,container:null,customKeys:null,extraKeys:null};a.defineOption("hintOptions",null)}); \ No newline at end of file +break}}},tg=w.signal=function(a,b){var c=a._handlers&&a._handlers[b];if(c)for(var d=Array.prototype.slice.call(arguments,2),e=0;eh||h>=b)return g+(b-f);g+=h-f,g+=c-g%c,f=h+1}},Jg=[""],Mg=function(a){a.select()};n?Mg=function(a){a.selectionStart=0,a.selectionEnd=a.value.length}:d&&(Mg=function(a){try{a.select()}catch(b){}}),[].indexOf&&(Ng=function(a,b){return a.indexOf(b)}),[].map&&(Og=function(a,b){return a.map(b)});var Zg,Sg=/[\u00df\u0590-\u05f4\u0600-\u06ff\u3040-\u309f\u30a0-\u30ff\u3400-\u4db5\u4e00-\u9fcc\uac00-\ud7af]/,Tg=w.isWordChar=function(a){return/\w/.test(a)||a>"\x80"&&(a.toUpperCase()!=a.toLowerCase()||Sg.test(a))},Wg=/[\u0300-\u036f\u0483-\u0489\u0591-\u05bd\u05bf\u05c1\u05c2\u05c4\u05c5\u05c7\u0610-\u061a\u064b-\u065e\u0670\u06d6-\u06dc\u06de-\u06e4\u06e7\u06e8\u06ea-\u06ed\u0711\u0730-\u074a\u07a6-\u07b0\u07eb-\u07f3\u0816-\u0819\u081b-\u0823\u0825-\u0827\u0829-\u082d\u0900-\u0902\u093c\u0941-\u0948\u094d\u0951-\u0955\u0962\u0963\u0981\u09bc\u09be\u09c1-\u09c4\u09cd\u09d7\u09e2\u09e3\u0a01\u0a02\u0a3c\u0a41\u0a42\u0a47\u0a48\u0a4b-\u0a4d\u0a51\u0a70\u0a71\u0a75\u0a81\u0a82\u0abc\u0ac1-\u0ac5\u0ac7\u0ac8\u0acd\u0ae2\u0ae3\u0b01\u0b3c\u0b3e\u0b3f\u0b41-\u0b44\u0b4d\u0b56\u0b57\u0b62\u0b63\u0b82\u0bbe\u0bc0\u0bcd\u0bd7\u0c3e-\u0c40\u0c46-\u0c48\u0c4a-\u0c4d\u0c55\u0c56\u0c62\u0c63\u0cbc\u0cbf\u0cc2\u0cc6\u0ccc\u0ccd\u0cd5\u0cd6\u0ce2\u0ce3\u0d3e\u0d41-\u0d44\u0d4d\u0d57\u0d62\u0d63\u0dca\u0dcf\u0dd2-\u0dd4\u0dd6\u0ddf\u0e31\u0e34-\u0e3a\u0e47-\u0e4e\u0eb1\u0eb4-\u0eb9\u0ebb\u0ebc\u0ec8-\u0ecd\u0f18\u0f19\u0f35\u0f37\u0f39\u0f71-\u0f7e\u0f80-\u0f84\u0f86\u0f87\u0f90-\u0f97\u0f99-\u0fbc\u0fc6\u102d-\u1030\u1032-\u1037\u1039\u103a\u103d\u103e\u1058\u1059\u105e-\u1060\u1071-\u1074\u1082\u1085\u1086\u108d\u109d\u135f\u1712-\u1714\u1732-\u1734\u1752\u1753\u1772\u1773\u17b7-\u17bd\u17c6\u17c9-\u17d3\u17dd\u180b-\u180d\u18a9\u1920-\u1922\u1927\u1928\u1932\u1939-\u193b\u1a17\u1a18\u1a56\u1a58-\u1a5e\u1a60\u1a62\u1a65-\u1a6c\u1a73-\u1a7c\u1a7f\u1b00-\u1b03\u1b34\u1b36-\u1b3a\u1b3c\u1b42\u1b6b-\u1b73\u1b80\u1b81\u1ba2-\u1ba5\u1ba8\u1ba9\u1c2c-\u1c33\u1c36\u1c37\u1cd0-\u1cd2\u1cd4-\u1ce0\u1ce2-\u1ce8\u1ced\u1dc0-\u1de6\u1dfd-\u1dff\u200c\u200d\u20d0-\u20f0\u2cef-\u2cf1\u2de0-\u2dff\u302a-\u302f\u3099\u309a\ua66f-\ua672\ua67c\ua67d\ua6f0\ua6f1\ua802\ua806\ua80b\ua825\ua826\ua8c4\ua8e0-\ua8f1\ua926-\ua92d\ua947-\ua951\ua980-\ua982\ua9b3\ua9b6-\ua9b9\ua9bc\uaa29-\uaa2e\uaa31\uaa32\uaa35\uaa36\uaa43\uaa4c\uaab0\uaab2-\uaab4\uaab7\uaab8\uaabe\uaabf\uaac1\uabe5\uabe8\uabed\udc00-\udfff\ufb1e\ufe00-\ufe0f\ufe20-\ufe26\uff9e\uff9f]/;Zg=document.createRange?function(a,b,c){var d=document.createRange();return d.setEnd(a,c),d.setStart(a,b),d}:function(a,b,c){var d=document.body.createTextRange();try{d.moveToElementText(a.parentNode)}catch(e){return d}return d.collapse(!0),d.moveEnd("character",c),d.moveStart("character",b),d},d&&11>e&&(bh=function(){try{return document.activeElement}catch(a){return document.body}});var lh,nh,ph,dh=w.rmClass=function(a,b){var c=a.className,d=ch(b).exec(c);if(d){var e=c.slice(d.index+d[0].length);a.className=c.slice(0,d.index)+(e?d[1]+e:"")}},eh=w.addClass=function(a,b){var c=a.className;ch(b).test(c)||(a.className+=(c?" ":"")+b)},hh=!1,kh=function(){if(d&&9>e)return!1;var a=Yg("div");return"draggable"in a||"dragDrop"in a}(),rh=w.splitLines=3!="\n\nb".split(/\n/).length?function(a){for(var b=0,c=[],d=a.length;d>=b;){var e=a.indexOf("\n",b);-1==e&&(e=a.length);var f=a.slice(b,"\r"==a.charAt(e-1)?e-1:e),g=f.indexOf("\r");-1!=g?(c.push(f.slice(0,g)),b+=g+1):(c.push(f),b=e+1)}return c}:function(a){return a.split(/\r\n?|\n/)},sh=window.getSelection?function(a){try{return a.selectionStart!=a.selectionEnd}catch(b){return!1}}:function(a){try{var b=a.ownerDocument.selection.createRange()}catch(c){}return b&&b.parentElement()==a?0!=b.compareEndPoints("StartToEnd",b):!1},th=function(){var a=Yg("div");return"oncopy"in a?!0:(a.setAttribute("oncopy","return;"),"function"==typeof a.oncopy)}(),uh=null,wh={3:"Enter",8:"Backspace",9:"Tab",13:"Enter",16:"Shift",17:"Ctrl",18:"Alt",19:"Pause",20:"CapsLock",27:"Esc",32:"Space",33:"PageUp",34:"PageDown",35:"End",36:"Home",37:"Left",38:"Up",39:"Right",40:"Down",44:"PrintScrn",45:"Insert",46:"Delete",59:";",61:"=",91:"Mod",92:"Mod",93:"Mod",107:"=",109:"-",127:"Delete",173:"-",186:";",187:"=",188:",",189:"-",190:".",191:"/",192:"`",219:"[",220:"\\",221:"]",222:"'",63232:"Up",63233:"Down",63234:"Left",63235:"Right",63272:"Delete",63273:"Home",63275:"End",63276:"PageUp",63277:"PageDown",63302:"Insert"};w.keyNames=wh,function(){for(var a=0;10>a;a++)wh[a+48]=wh[a+96]=String(a);for(var a=65;90>=a;a++)wh[a]=String.fromCharCode(a);for(var a=1;12>=a;a++)wh[a+111]=wh[a+63235]="F"+a}();var Gh,Lh=function(){function c(c){return 247>=c?a.charAt(c):c>=1424&&1524>=c?"R":c>=1536&&1773>=c?b.charAt(c-1536):c>=1774&&2220>=c?"r":c>=8192&&8203>=c?"w":8204==c?"b":"L"}function j(a,b,c){this.level=a,this.from=b,this.to=c}var a="bbbbbbbbbtstwsbbbbbbbbbbbbbbssstwNN%%%NNNNNN,N,N1111111111NNNNNNNLLLLLLLLLLLLLLLLLLLLLLLLLLNNNNNNLLLLLLLLLLLLLLLLLLLLLLLLLLNNNNbbbbbbsbbbbbbbbbbbbbbbbbbbbbbbbbb,N%%%%NNNNLNNNNN%%11NLNNN1LNNNNNLLLLLLLLLLLLLLLLLLLLLLLNLLLLLLLLLLLLLLLLLLLLLLLLLLLLLLLN",b="rrrrrrrrrrrr,rNNmmmmmmrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrmmmmmmmmmmmmmmrrrrrrrnnnnnnnnnn%nnrrrmrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrrmmmmmmmmmmmmmmmmmmmNmmmm",d=/[\u0590-\u05f4\u0600-\u06ff\u0700-\u08ac]/,e=/[stwN]/,f=/[LRr]/,g=/[Lb1n]/,h=/[1n]/,i="L";return function(a){if(!d.test(a))return!1;for(var m,b=a.length,k=[],l=0;b>l;++l)k.push(m=c(a.charCodeAt(l)));for(var l=0,n=i;b>l;++l){var m=k[l];"m"==m?k[l]=n:n=m}for(var l=0,o=i;b>l;++l){var m=k[l];"1"==m&&"r"==o?k[l]="n":f.test(m)&&(o=m,"r"==m&&(k[l]="R"))}for(var l=1,n=k[0];b-1>l;++l){var m=k[l];"+"==m&&"1"==n&&"1"==k[l+1]?k[l]="1":","!=m||n!=k[l+1]||"1"!=n&&"n"!=n||(k[l]=n),n=m}for(var l=0;b>l;++l){var m=k[l];if(","==m)k[l]="N";else if("%"==m){for(var p=l+1;b>p&&"%"==k[p];++p);for(var q=l&&"!"==k[l-1]||b>p&&"1"==k[p]?"1":"N",r=l;p>r;++r)k[r]=q;l=p-1}}for(var l=0,o=i;b>l;++l){var m=k[l];"L"==o&&"1"==m?k[l]="L":f.test(m)&&(o=m)}for(var l=0;b>l;++l)if(e.test(k[l])){for(var p=l+1;b>p&&e.test(k[p]);++p);for(var s="L"==(l?k[l-1]:i),t="L"==(b>p?k[p]:i),q=s||t?"L":"R",r=l;p>r;++r)k[r]=q;l=p-1}for(var v,u=[],l=0;b>l;)if(g.test(k[l])){var w=l;for(++l;b>l&&g.test(k[l]);++l);u.push(new j(0,w,l))}else{var x=l,y=u.length;for(++l;b>l&&"L"!=k[l];++l);for(var r=x;l>r;)if(h.test(k[r])){r>x&&u.splice(y,0,new j(1,x,r));var z=r;for(++r;l>r&&h.test(k[r]);++r);u.splice(y,0,new j(2,z,r)),x=r}else++r;l>x&&u.splice(y,0,new j(1,x,l))}return 1==u[0].level&&(v=a.match(/^\s+/))&&(u[0].from=v[0].length,u.unshift(new j(0,0,v[0].length))),1==Lg(u).level&&(v=a.match(/\s+$/))&&(Lg(u).to-=v[0].length,u.push(new j(0,b-v[0].length,b))),u[0].level!=Lg(u).level&&u.push(new j(u[0].level,b,b)),u}}();return w.version="4.8.0",w}),function(a){"object"==typeof exports&&"object"==typeof module?a(require("../../lib/codemirror")):"function"==typeof define&&define.amd?define(["../../lib/codemirror"],a):a(CodeMirror)}(function(a){"use strict";a.defineMode("javascript",function(b,c){function m(a){for(var c,b=!1,d=!1;null!=(c=a.next());){if(!b){if("/"==c&&!d)return;"["==c?d=!0:d&&"]"==c&&(d=!1)}b=!b&&"\\"==c}}function p(a,b,c){return n=a,o=c,b}function q(a,b){var c=a.next();if('"'==c||"'"==c)return b.tokenize=r(c),b.tokenize(a,b);if("."==c&&a.match(/^\d+(?:[eE][+\-]?\d+)?/))return p("number","number");if("."==c&&a.match(".."))return p("spread","meta");if(/[\[\]{}\(\),;\:\.]/.test(c))return p(c);if("="==c&&a.eat(">"))return p("=>","operator");if("0"==c&&a.eat(/x/i))return a.eatWhile(/[\da-f]/i),p("number","number");if(/\d/.test(c))return a.match(/^\d*(?:\.\d*)?(?:[eE][+\-]?\d+)?/),p("number","number");if("/"==c)return a.eat("*")?(b.tokenize=s,s(a,b)):a.eat("/")?(a.skipToEnd(),p("comment","comment")):"operator"==b.lastType||"keyword c"==b.lastType||"sof"==b.lastType||/^[\[{}\(,;:]$/.test(b.lastType)?(m(a),a.eatWhile(/[gimy]/),p("regexp","string-2")):(a.eatWhile(k),p("operator","operator",a.current()));if("`"==c)return b.tokenize=t,t(a,b);if("#"==c)return a.skipToEnd(),p("error","error");if(k.test(c))return a.eatWhile(k),p("operator","operator",a.current());if(i.test(c)){a.eatWhile(i);var d=a.current(),e=j.propertyIsEnumerable(d)&&j[d];return e&&"."!=b.lastType?p(e.type,e.style,d):p("variable","variable",d)}}function r(a){return function(b,c){var e,d=!1;if(f&&"@"==b.peek()&&b.match(l))return c.tokenize=q,p("jsonld-keyword","meta");for(;null!=(e=b.next())&&(e!=a||d);)d=!d&&"\\"==e;return d||(c.tokenize=q),p("string","string")}}function s(a,b){for(var d,c=!1;d=a.next();){if("/"==d&&c){b.tokenize=q;break}c="*"==d}return p("comment","comment")}function t(a,b){for(var d,c=!1;null!=(d=a.next());){if(!c&&("`"==d||"$"==d&&a.eat("{"))){b.tokenize=q;break}c=!c&&"\\"==d}return p("quasi","string-2",a.current())}function v(a,b){b.fatArrowAt&&(b.fatArrowAt=null);var c=a.string.indexOf("=>",a.start);if(!(0>c)){for(var d=0,e=!1,f=c-1;f>=0;--f){var g=a.string.charAt(f),h=u.indexOf(g);if(h>=0&&3>h){if(!d){++f;break}if(0==--d)break}else if(h>=3&&6>h)++d;else if(i.test(g))e=!0;else if(e&&!d){++f;break}}e&&!d&&(b.fatArrowAt=f)}}function x(a,b,c,d,e,f){this.indented=a,this.column=b,this.type=c,this.prev=e,this.info=f,null!=d&&(this.align=d)}function y(a,b){for(var c=a.localVars;c;c=c.next)if(c.name==b)return!0;for(var d=a.context;d;d=d.prev)for(var c=d.vars;c;c=c.next)if(c.name==b)return!0}function z(a,b,c,d,e){var f=a.cc;for(A.state=a,A.stream=e,A.marked=null,A.cc=f,A.style=b,a.lexical.hasOwnProperty("align")||(a.lexical.align=!0);;){var h=f.length?f.pop():g?L:K;if(h(c,d)){for(;f.length&&f[f.length-1].lex;)f.pop()();return A.marked?A.marked:"variable"==c&&y(a,d)?"variable-2":b}}}function B(){for(var a=arguments.length-1;a>=0;a--)A.cc.push(arguments[a])}function C(){return B.apply(null,arguments),!0}function D(a){function b(b){for(var c=b;c;c=c.next)if(c.name==a)return!0;return!1}var d=A.state;if(d.context){if(A.marked="def",b(d.localVars))return;d.localVars={name:a,next:d.localVars}}else{if(b(d.globalVars))return;c.globalVars&&(d.globalVars={name:a,next:d.globalVars})}}function F(){A.state.context={prev:A.state.context,vars:A.state.localVars},A.state.localVars=E}function G(){A.state.localVars=A.state.context.vars,A.state.context=A.state.context.prev}function H(a,b){var c=function(){var c=A.state,d=c.indented;if("stat"==c.lexical.type)d=c.lexical.indented;else for(var e=c.lexical;e&&")"==e.type&&e.align;e=e.prev)d=e.indented;c.lexical=new x(d,A.stream.column(),a,null,c.lexical,b)};return c.lex=!0,c}function I(){var a=A.state;a.lexical.prev&&(")"==a.lexical.type&&(a.indented=a.lexical.indented),a.lexical=a.lexical.prev)}function J(a){function b(c){return c==a?C():";"==a?B():C(b)}return b}function K(a,b){return"var"==a?C(H("vardef",b.length),eb,J(";"),I):"keyword a"==a?C(H("form"),L,K,I):"keyword b"==a?C(H("form"),K,I):"{"==a?C(H("}"),bb,I):";"==a?C():"if"==a?("else"==A.state.lexical.info&&A.state.cc[A.state.cc.length-1]==I&&A.state.cc.pop()(),C(H("form"),L,K,I,jb)):"function"==a?C(pb):"for"==a?C(H("form"),kb,K,I):"variable"==a?C(H("stat"),W):"switch"==a?C(H("form"),L,H("}","switch"),J("{"),bb,I,I):"case"==a?C(L,J(":")):"default"==a?C(J(":")):"catch"==a?C(H("form"),F,J("("),qb,J(")"),K,I,G):"module"==a?C(H("form"),F,vb,G,I):"class"==a?C(H("form"),rb,I):"export"==a?C(H("form"),wb,I):"import"==a?C(H("form"),xb,I):B(H("stat"),L,J(";"),I)}function L(a){return N(a,!1)}function M(a){return N(a,!0)}function N(a,b){if(A.state.fatArrowAt==A.stream.start){var c=b?V:U;if("("==a)return C(F,H(")"),_(fb,")"),I,J("=>"),c,G);if("variable"==a)return B(F,fb,J("=>"),c,G)}var d=b?R:Q;return w.hasOwnProperty(a)?C(d):"function"==a?C(pb,d):"keyword c"==a?C(b?P:O):"("==a?C(H(")"),O,Cb,J(")"),I,d):"operator"==a||"spread"==a?C(b?M:L):"["==a?C(H("]"),Ab,I,d):"{"==a?ab(Y,"}",null,d):"quasi"==a?B(S,d):C()}function O(a){return a.match(/[;\}\)\],]/)?B():B(L)}function P(a){return a.match(/[;\}\)\],]/)?B():B(M)}function Q(a,b){return","==a?C(L):R(a,b,!1)}function R(a,b,c){var d=0==c?Q:R,e=0==c?L:M;return"=>"==a?C(F,c?V:U,G):"operator"==a?/\+\+|--/.test(b)?C(d):"?"==b?C(L,J(":"),e):C(e):"quasi"==a?B(S,d):";"!=a?"("==a?ab(M,")","call",d):"."==a?C(X,d):"["==a?C(H("]"),O,J("]"),I,d):void 0:void 0}function S(a,b){return"quasi"!=a?B():"${"!=b.slice(b.length-2)?C(S):C(L,T)}function T(a){return"}"==a?(A.marked="string-2",A.state.tokenize=t,C(S)):void 0}function U(a){return v(A.stream,A.state),B("{"==a?K:L)}function V(a){return v(A.stream,A.state),B("{"==a?K:M)}function W(a){return":"==a?C(I,K):B(Q,J(";"),I)}function X(a){return"variable"==a?(A.marked="property",C()):void 0}function Y(a,b){return"variable"==a||"keyword"==A.style?(A.marked="property","get"==b||"set"==b?C(Z):C($)):"number"==a||"string"==a?(A.marked=f?"property":A.style+" property",C($)):"jsonld-keyword"==a?C($):"["==a?C(L,J("]"),$):void 0}function Z(a){return"variable"!=a?B($):(A.marked="property",C(pb))}function $(a){return":"==a?C(M):"("==a?B(pb):void 0}function _(a,b){function c(d){if(","==d){var e=A.state.lexical;return"call"==e.info&&(e.pos=(e.pos||0)+1),C(a,c)}return d==b?C():C(J(b))}return function(d){return d==b?C():B(a,c)}}function ab(a,b,c){for(var d=3;d!?|~^]/,l=/^@(context|id|value|language|type|container|list|set|reverse|index|base|vocab|graph)"/,u="([{}])",w={atom:!0,number:!0,variable:!0,string:!0,regexp:!0,"this":!0,"jsonld-keyword":!0},A={state:null,column:null,marked:null,cc:null},E={name:"this",next:{name:"arguments"}};return I.lex=!0,{startState:function(a){var b={tokenize:q,lastType:"sof",cc:[],lexical:new x((a||0)-d,0,"block",!1),localVars:c.localVars,context:c.localVars&&{vars:c.localVars},indented:0};return c.globalVars&&"object"==typeof c.globalVars&&(b.globalVars=c.globalVars),b},token:function(a,b){if(a.sol()&&(b.lexical.hasOwnProperty("align")||(b.lexical.align=!1),b.indented=a.indentation(),v(a,b)),b.tokenize!=s&&a.eatSpace())return null;var c=b.tokenize(a,b);return"comment"==n?c:(b.lastType="operator"!=n||"++"!=o&&"--"!=o?n:"incdec",z(b,c,n,o,a))},indent:function(b,f){if(b.tokenize==s)return a.Pass;if(b.tokenize!=q)return 0;var g=f&&f.charAt(0),h=b.lexical;if(!/^\s*else\b/.test(f))for(var i=b.cc.length-1;i>=0;--i){var j=b.cc[i];if(j==I)h=h.prev;else if(j!=jb)break}"stat"==h.type&&"}"==g&&(h=h.prev),e&&")"==h.type&&"stat"==h.prev.type&&(h=h.prev);var k=h.type,l=g==k;return"vardef"==k?h.indented+("operator"==b.lastType||","==b.lastType?h.info+1:0):"form"==k&&"{"==g?h.indented:"form"==k?h.indented+d:"stat"==k?h.indented+("operator"==b.lastType||","==b.lastType?e||d:0):"switch"!=h.info||l||0==c.doubleIndentSwitch?h.align?h.column+(l?0:1):h.indented+(l?0:d):h.indented+(/^(?:case|default)\b/.test(f)?d:2*d)},electricInput:/^\s*(?:case .*?:|default:|\{|\})$/,blockCommentStart:g?null:"/*",blockCommentEnd:g?null:"*/",lineComment:g?null:"//",fold:"brace",helperType:g?"json":"javascript",jsonldMode:f,jsonMode:g}}),a.registerHelper("wordChars","javascript",/[\w$]/),a.defineMIME("text/javascript","javascript"),a.defineMIME("text/ecmascript","javascript"),a.defineMIME("application/javascript","javascript"),a.defineMIME("application/x-javascript","javascript"),a.defineMIME("application/ecmascript","javascript"),a.defineMIME("application/json",{name:"javascript",json:!0}),a.defineMIME("application/x-json",{name:"javascript",json:!0}),a.defineMIME("application/ld+json",{name:"javascript",jsonld:!0}),a.defineMIME("text/typescript",{name:"javascript",typescript:!0}),a.defineMIME("application/typescript",{name:"javascript",typescript:!0})}),function(a){"object"==typeof exports&&"object"==typeof module?a(require("../../lib/codemirror")):"function"==typeof define&&define.amd?define(["../../lib/codemirror"],a):a(CodeMirror)}(function(a){"use strict";a.defineMode("xml",function(b,c){function k(a,b){function c(c){return b.tokenize=c,c(a,b)}var d=a.next();if("<"==d)return a.eat("!")?a.eat("[")?a.match("CDATA[")?c(n("atom","]]>")):null:a.match("--")?c(n("comment","-->")):a.match("DOCTYPE",!0,!0)?(a.eatWhile(/[\w\._\-]/),c(o(1))):null:a.eat("?")?(a.eatWhile(/[\w\._\-]/),b.tokenize=n("meta","?>"),"meta"):(i=a.eat("/")?"closeTag":"openTag",b.tokenize=l,"tag bracket");if("&"==d){var e;return e=a.eat("#")?a.eat("x")?a.eatWhile(/[a-fA-F\d]/)&&a.eat(";"):a.eatWhile(/[\d]/)&&a.eat(";"):a.eatWhile(/[\w\.\-:]/)&&a.eat(";"),e?"atom":"error"}return a.eatWhile(/[^&<]/),null}function l(a,b){var c=a.next();if(">"==c||"/"==c&&a.eat(">"))return b.tokenize=k,i=">"==c?"endTag":"selfcloseTag","tag bracket";if("="==c)return i="equals",null;if("<"==c){b.tokenize=k,b.state=s,b.tagName=b.tagStart=null;var d=b.tokenize(a,b);return d?d+" tag error":"tag error"}return/[\'\"]/.test(c)?(b.tokenize=m(c),b.stringStartCol=a.column(),b.tokenize(a,b)):(a.match(/^[^\s\u00a0=<>\"\']*[^\s\u00a0=<>\"\'\/]/),"word")}function m(a){var b=function(b,c){for(;!b.eol();)if(b.next()==a){c.tokenize=l;break}return"string"};return b.isInAttribute=!0,b}function n(a,b){return function(c,d){for(;!c.eol();){if(c.match(b)){d.tokenize=k;break}c.next()}return a}}function o(a){return function(b,c){for(var d;null!=(d=b.next());){if("<"==d)return c.tokenize=o(a+1),c.tokenize(b,c);if(">"==d){if(1==a){c.tokenize=k;break}return c.tokenize=o(a-1),c.tokenize(b,c)}}return"meta"}}function p(a,b,c){this.prev=a.context,this.tagName=b,this.indent=a.indented,this.startOfLine=c,(g.doNotIndent.hasOwnProperty(b)||a.context&&a.context.noIndent)&&(this.noIndent=!0)}function q(a){a.context&&(a.context=a.context.prev)}function r(a,b){for(var c;;){if(!a.context)return;if(c=a.context.tagName,!g.contextGrabbers.hasOwnProperty(c)||!g.contextGrabbers[c].hasOwnProperty(b))return;q(a)}}function s(a,b,c){return"openTag"==a?(c.tagStart=b.column(),t):"closeTag"==a?u:s}function t(a,b,c){return"word"==a?(c.tagName=b.current(),j="tag",x):(j="error",t)}function u(a,b,c){if("word"==a){var d=b.current();return c.context&&c.context.tagName!=d&&g.implicitlyClosed.hasOwnProperty(c.context.tagName)&&q(c),c.context&&c.context.tagName==d?(j="tag",v):(j="tag error",w)}return j="error",w}function v(a,b,c){return"endTag"!=a?(j="error",v):(q(c),s)}function w(a,b,c){return j="error",v(a,b,c)}function x(a,b,c){if("word"==a)return j="attribute",y;if("endTag"==a||"selfcloseTag"==a){var d=c.tagName,e=c.tagStart;return c.tagName=c.tagStart=null,"selfcloseTag"==a||g.autoSelfClosers.hasOwnProperty(d)?r(c,d):(r(c,d),c.context=new p(c,d,e==c.indented)),s}return j="error",x}function y(a,b,c){return"equals"==a?z:(g.allowMissing||(j="error"),x(a,b,c))}function z(a,b,c){return"string"==a?A:"word"==a&&g.allowUnquoted?(j="string",x):(j="error",x(a,b,c))}function A(a,b,c){return"string"==a?A:x(a,b,c)}var d=b.indentUnit,e=c.multilineTagIndentFactor||1,f=c.multilineTagIndentPastTag;null==f&&(f=!0);var i,j,g=c.htmlMode?{autoSelfClosers:{area:!0,base:!0,br:!0,col:!0,command:!0,embed:!0,frame:!0,hr:!0,img:!0,input:!0,keygen:!0,link:!0,meta:!0,param:!0,source:!0,track:!0,wbr:!0,menuitem:!0},implicitlyClosed:{dd:!0,li:!0,optgroup:!0,option:!0,p:!0,rp:!0,rt:!0,tbody:!0,td:!0,tfoot:!0,th:!0,tr:!0},contextGrabbers:{dd:{dd:!0,dt:!0},dt:{dd:!0,dt:!0},li:{li:!0},option:{option:!0,optgroup:!0},optgroup:{optgroup:!0},p:{address:!0,article:!0,aside:!0,blockquote:!0,dir:!0,div:!0,dl:!0,fieldset:!0,footer:!0,form:!0,h1:!0,h2:!0,h3:!0,h4:!0,h5:!0,h6:!0,header:!0,hgroup:!0,hr:!0,menu:!0,nav:!0,ol:!0,p:!0,pre:!0,section:!0,table:!0,ul:!0},rp:{rp:!0,rt:!0},rt:{rp:!0,rt:!0},tbody:{tbody:!0,tfoot:!0},td:{td:!0,th:!0},tfoot:{tbody:!0},th:{td:!0,th:!0},thead:{tbody:!0,tfoot:!0},tr:{tr:!0}},doNotIndent:{pre:!0},allowUnquoted:!0,allowMissing:!0,caseFold:!0}:{autoSelfClosers:{},implicitlyClosed:{},contextGrabbers:{},doNotIndent:{},allowUnquoted:!1,allowMissing:!1,caseFold:!1},h=c.alignCDATA;return{startState:function(){return{tokenize:k,state:s,indented:0,tagName:null,tagStart:null,context:null}},token:function(a,b){if(!b.tagName&&a.sol()&&(b.indented=a.indentation()),a.eatSpace())return null;i=null;var c=b.tokenize(a,b);return(c||i)&&"comment"!=c&&(j=null,b.state=b.state(i||c,a,b),j&&(c="error"==j?c+" error":j)),c},indent:function(b,c,i){var j=b.context;if(b.tokenize.isInAttribute)return b.tagStart==b.indented?b.stringStartCol+1:b.indented+d;if(j&&j.noIndent)return a.Pass;if(b.tokenize!=l&&b.tokenize!=k)return i?i.match(/^(\s*)/)[0].length:0;if(b.tagName)return f?b.tagStart+b.tagName.length+2:b.tagStart+d*e;if(h&&/$/,blockCommentStart:"",configuration:c.htmlMode?"html":"xml",helperType:c.htmlMode?"html":"xml"}}),a.defineMIME("text/xml","xml"),a.defineMIME("application/xml","xml"),a.mimeModes.hasOwnProperty("text/html")||a.defineMIME("text/html",{name:"xml",htmlMode:!0})}),function(a){"object"==typeof exports&&"object"==typeof module?a(require("../../lib/codemirror")):"function"==typeof define&&define.amd?define(["../../lib/codemirror"],a):a(CodeMirror)}(function(a){"use strict";a.registerHelper("fold","brace",function(b,c){function h(f){for(var h=c.ch,i=0;;){var j=0>=h?-1:e.lastIndexOf(f,h-1);if(-1!=j){if(1==i&&j=o;++o)for(var p=b.getLine(o),q=o==d?f:0;;){var r=p.indexOf(i,q),s=p.indexOf(j,q);if(0>r&&(r=p.length),0>s&&(s=p.length),q=Math.min(r,s),q==p.length)break;if(b.getTokenTypeAt(a.Pos(o,q+1))==g)if(q==r)++k;else if(!--k){m=o,n=q;break a}++q}if(null!=m&&(d!=m||n!=f))return{from:a.Pos(d,f),to:a.Pos(m,n)}}}),a.registerHelper("fold","import",function(b,c){function d(c){if(cb.lastLine())return null;var d=b.getTokenAt(a.Pos(c,1));if(/\S/.test(d.string)||(d=b.getTokenAt(a.Pos(c,d.end+1))),"keyword"!=d.type||"import"!=d.string)return null;for(var e=c,f=Math.min(b.lastLine(),c+10);f>=e;++e){var g=b.getLine(e),h=g.indexOf(";");if(-1!=h)return{startCh:d.end,end:a.Pos(e,h)}}}var f,c=c.line,e=d(c);if(!e||d(c-1)||(f=d(c-2))&&f.end.line==c-1)return null;for(var g=e.end;;){var h=d(g.line+1);if(null==h)break;g=h.end}return{from:b.clipPos(a.Pos(c,e.startCh+1)),to:g}}),a.registerHelper("fold","include",function(b,c){function d(c){if(cb.lastLine())return null;var d=b.getTokenAt(a.Pos(c,1));return/\S/.test(d.string)||(d=b.getTokenAt(a.Pos(c,d.end+1))),"meta"==d.type&&"#include"==d.string.slice(0,8)?d.start+8:void 0}var c=c.line,e=d(c);if(null==e||null!=d(c-1))return null;for(var f=c;;){var g=d(f+1);if(null==g)break;++f}return{from:a.Pos(c,e+1),to:b.clipPos(a.Pos(f))}})}),function(a){"object"==typeof exports&&"object"==typeof module?a(require("../../lib/codemirror")):"function"==typeof define&&define.amd?define(["../../lib/codemirror"],a):a(CodeMirror)}(function(a){"use strict";a.registerGlobalHelper("fold","comment",function(a){return a.blockCommentStart&&a.blockCommentEnd},function(b,c){var d=b.getModeAt(c),e=d.blockCommentStart,f=d.blockCommentEnd;if(e&&f){for(var i,g=c.line,h=b.getLine(g),j=c.ch,k=0;;){var l=0>=j?-1:h.lastIndexOf(e,j-1);if(-1!=l){if(1==k&&l=q;++q)for(var r=b.getLine(q),s=q==g?i:0;;){var t=r.indexOf(e,s),u=r.indexOf(f,s);if(0>t&&(t=r.length),0>u&&(u=r.length),s=Math.min(t,u),s==r.length)break;if(s==t)++m;else if(!--m){o=q,p=s;break a}++s}if(null!=o&&(g!=o||p!=i))return{from:a.Pos(g,i),to:a.Pos(o,p)}}})}),function(a){"object"==typeof exports&&"object"==typeof module?a(require("../../lib/codemirror")):"function"==typeof define&&define.amd?define(["../../lib/codemirror"],a):a(CodeMirror)}(function(a){"use strict";function b(b,d,f,g){function j(a){var c=h(b,d);if(!c||c.to.line-c.from.lineb.firstLine();)d=a.Pos(d.line-1,0),k=j(!1);if(k&&!k.cleared&&"unfold"!==g){var l=c(b,f);a.on(l,"mousedown",function(b){m.clear(),a.e_preventDefault(b)});var m=b.markText(k.from,k.to,{replacedWith:l,clearOnEnter:!0,__isFold:!0});m.on("clear",function(c,d){a.signal(b,"unfold",b,c,d)}),a.signal(b,"fold",b,k.from,k.to)}}function c(a,b){var c=e(a,b,"widget");if("string"==typeof c){var d=document.createTextNode(c);c=document.createElement("span"),c.appendChild(d),c.className="CodeMirror-foldmarker"}return c}function e(a,b,c){if(b&&void 0!==b[c])return b[c];var e=a.options.foldOptions;return e&&void 0!==e[c]?e[c]:d[c]}a.newFoldFunction=function(a,c){return function(d,e){b(d,e,{rangeFinder:a,widget:c})}},a.defineExtension("foldCode",function(a,c,d){b(this,a,c,d)}),a.defineExtension("isFolded",function(a){for(var b=this.findMarksAt(a),c=0;c=c;c++)b.foldCode(a.Pos(c,0),null,"fold")})},a.commands.unfoldAll=function(b){b.operation(function(){for(var c=b.firstLine(),d=b.lastLine();d>=c;c++)b.foldCode(a.Pos(c,0),null,"unfold")})},a.registerHelper("fold","combine",function(){var a=Array.prototype.slice.call(arguments,0);return function(b,c){for(var d=0;d20||b.from-c.to>20?h(a):a.operation(function(){c.fromb.to&&(g(a,b.to,c.to),b.to=c.to)})},c.updateViewportTimeSpan||400)}function l(a,b){var c=a.state.foldGutter,d=b.line;d>=c.from&&dk&&!(f(k+1,h,l)<=i);)++k,h=l,l=b.getLine(k+2);return{from:a.Pos(c.line,g.length),to:a.Pos(k,b.getLine(k).length)}})}),function(a){"object"==typeof exports&&"object"==typeof module?a(require("../../lib/codemirror")):"function"==typeof define&&define.amd?define(["../../lib/codemirror"],a):a(CodeMirror)}(function(a){function e(a,b,e,g){var h=a.getLineHandle(b.line),i=b.ch-1,j=i>=0&&d[h.text.charAt(i)]||d[h.text.charAt(++i)];if(!j)return null;var k=">"==j.charAt(1)?1:-1;if(e&&k>0!=(i==b.ch))return null;var l=a.getTokenTypeAt(c(b.line,i+1)),m=f(a,c(b.line,i+(k>0?1:0)),k,l||null,g);return null==m?null:{from:c(b.line,i),to:m&&m.pos,match:m&&m.ch==j.charAt(0),forward:k>0}}function f(a,b,e,f,g){for(var h=g&&g.maxScanLineLength||1e4,i=g&&g.maxScanLines||1e3,j=[],k=g&&g.bracketRegex?g.bracketRegex:/[(){}[\]]/,l=e>0?Math.min(b.line+i,a.lastLine()+1):Math.max(a.firstLine()-1,b.line-i),m=b.line;m!=l;m+=e){var n=a.getLine(m);if(n){var o=e>0?0:n.length-1,p=e>0?n.length:-1;if(!(n.length>h))for(m==b.line&&(o=b.ch-(0>e?1:0));o!=p;o+=e){var q=n.charAt(o);if(k.test(q)&&(void 0===f||a.getTokenTypeAt(c(m,o+1))==f)){var r=d[q];if(">"==r.charAt(1)==e>0)j.push(q);else{if(!j.length)return{pos:c(m,o),ch:q};j.pop()}}}}}return m-e==(e>0?a.lastLine():a.firstLine())?!1:null}function g(a,d,f){for(var g=a.state.matchBrackets.maxHighlightLineLength||1e3,h=[],i=a.listSelections(),j=0;j",")":"(<","[":"]>","]":"[<","{":"}>","}":"{<"},h=null;a.defineOption("matchBrackets",!1,function(b,c,d){d&&d!=a.Init&&b.off("cursorActivity",i),c&&(b.state.matchBrackets="object"==typeof c?c:{},b.on("cursorActivity",i))}),a.defineExtension("matchBrackets",function(){g(this,!0)}),a.defineExtension("findMatchingBracket",function(a,b,c){return e(this,a,b,c)}),a.defineExtension("scanForBracket",function(a,b,c,d){return f(this,a,b,c,d)})}),function(a){"object"==typeof exports&&"object"==typeof module?a(require("../../lib/codemirror")):"function"==typeof define&&define.amd?define(["../../lib/codemirror"],a):a(CodeMirror)}(function(a){"use strict";function d(a,b){this.cm=a,this.options=this.buildOptions(b),this.widget=this.onClose=null}function e(a){return"string"==typeof a?a:a.text}function f(a,b){function f(a,d){var f;f="string"!=typeof d?function(a){return d(a,b)}:c.hasOwnProperty(d)?c[d]:d,e[a]=f}var c={Up:function(){b.moveFocus(-1)},Down:function(){b.moveFocus(1)},PageUp:function(){b.moveFocus(-b.menuSize()+1,!0)},PageDown:function(){b.moveFocus(b.menuSize()-1,!0)},Home:function(){b.setFocus(0)},End:function(){b.setFocus(b.length-1)},Enter:b.pick,Tab:b.pick,Esc:b.close},d=a.options.customKeys,e=d?{}:c;if(d)for(var g in d)d.hasOwnProperty(g)&&f(g,d[g]);var h=a.options.extraKeys;if(h)for(var g in h)h.hasOwnProperty(g)&&f(g,h[g]);return e}function g(a,b){for(;b&&b!=a;){if("LI"===b.nodeName.toUpperCase()&&b.parentNode==a)return b;b=b.parentNode}}function h(d,h){this.completion=d,this.data=h;var i=this,j=d.cm,k=this.hints=document.createElement("ul");k.className="CodeMirror-hints",this.selectedHint=h.selectedHint||0;for(var l=h.list,m=0;m0){var y=w.bottom-w.top,z=q.top-(q.bottom-w.top);if(z-y>0)k.style.top=(s=q.top-y)+"px",t=!1;else if(y>v){k.style.height=v-5+"px",k.style.top=(s=q.bottom-w.top)+"px";var A=j.getCursor();h.from.ch!=A.ch&&(q=j.cursorCoords(A),k.style.left=(r=q.left)+"px",w=k.getBoundingClientRect())}}var B=w.left-u;if(B>0&&(w.right-w.left>u&&(k.style.width=u-5+"px",B-=w.right-w.left-u),k.style.left=(r=q.left-B)+"px"),j.addKeyMap(this.keyMap=f(d,{moveFocus:function(a,b){i.changeActive(i.selectedHint+a,b)},setFocus:function(a){i.changeActive(a)},menuSize:function(){return i.screenAmount()},length:l.length,close:function(){d.close()},pick:function(){i.pick()},data:h})),d.options.closeOnUnfocus){var C;j.on("blur",this.onBlur=function(){C=setTimeout(function(){d.close()},100)}),j.on("focus",this.onFocus=function(){clearTimeout(C)})}var D=j.getScrollInfo();return j.on("scroll",this.onScroll=function(){var a=j.getScrollInfo(),b=j.getWrapperElement().getBoundingClientRect(),c=s+D.top-a.top,e=c-(window.pageYOffset||(document.documentElement||document.body).scrollTop);return t||(e+=k.offsetHeight),e<=b.top||e>=b.bottom?d.close():(k.style.top=c+"px",k.style.left=r+D.left-a.left+"px",void 0)}),a.on(k,"dblclick",function(a){var b=g(k,a.target||a.srcElement);b&&null!=b.hintId&&(i.changeActive(b.hintId),i.pick())}),a.on(k,"click",function(a){var b=g(k,a.target||a.srcElement);b&&null!=b.hintId&&(i.changeActive(b.hintId),d.options.completeOnSingleClick&&i.pick())}),a.on(k,"mousedown",function(){setTimeout(function(){j.focus()},20)}),a.signal(h,"select",l[0],k.firstChild),!0}var b="CodeMirror-hint",c="CodeMirror-hint-active";a.showHint=function(a,b,c){if(!b)return a.showHint(c);c&&c.async&&(b.async=!0);var d={hint:b};if(c)for(var e in c)d[e]=c[e];return a.showHint(d)},a.defineExtension("showHint",function(b){if(!(this.listSelections().length>1||this.somethingSelected())){this.state.completionActive&&this.state.completionActive.close();var c=this.state.completionActive=new d(this,b),e=c.options.hint;if(e)return a.signal(this,"startCompletion",this),e.async?(e(this,function(a){c.showHints(a)},c.options),void 0):c.showHints(e(this,c.options))}}),d.prototype={close:function(){this.active()&&(this.cm.state.completionActive=null,this.widget&&this.widget.close(),this.onClose&&this.onClose(),a.signal(this.cm,"endCompletion",this.cm))},active:function(){return this.cm.state.completionActive==this},pick:function(b,c){var d=b.list[c];d.hint?d.hint(this.cm,b,d):this.cm.replaceRange(e(d),d.from||b.from,d.to||b.to,"complete"),a.signal(b,"pick",d),this.close()},showHints:function(a){return a&&a.list.length&&this.active()?(this.options.completeSingle&&1==a.list.length?this.pick(a,0):this.showWidget(a),void 0):this.close()},showWidget:function(b){function l(){e||(e=!0,d.close(),d.cm.off("cursorActivity",p),b&&a.signal(b,"close"))}function m(){if(!e){a.signal(b,"update");var c=d.options.hint;c.async?c(d.cm,n,d.options):n(c(d.cm,d.options))}}function n(a){if(b=a,!e){if(!b||!b.list.length)return l();d.widget&&d.widget.close(),d.widget=new h(d,b)}}function o(){c&&(k(c),c=0)}function p(){o();var a=d.cm.getCursor(),b=d.cm.getLine(a.line);a.line!=g.line||b.length-a.ch!=i-g.ch||a.ch=this.data.list.length?b=d?this.data.list.length-1:0:0>b&&(b=d?0:this.data.list.length-1),this.selectedHint!=b){var e=this.hints.childNodes[this.selectedHint];e.className=e.className.replace(" "+c,""),e=this.hints.childNodes[this.selectedHint=b],e.className+=" "+c,e.offsetTopthis.hints.scrollTop+this.hints.clientHeight&&(this.hints.scrollTop=e.offsetTop+e.offsetHeight-this.hints.clientHeight+3),a.signal(this.data,"select",this.data.list[this.selectedHint],e)}},screenAmount:function(){return Math.floor(this.hints.clientHeight/this.hints.firstChild.offsetHeight)||1}},a.registerHelper("hint","auto",function(b,c){var e,d=b.getHelpers(b.getCursor(),"hint");if(d.length)for(var f=0;f,]/,closeOnUnfocus:!0,completeOnSingleClick:!1,container:null,customKeys:null,extraKeys:null};a.defineOption("hintOptions",null)}),function(a){"object"==typeof exports&&"object"==typeof module?a(require("../../lib/codemirror")):"function"==typeof define&&define.amd?define(["../../lib/codemirror"],a):a(CodeMirror)}(function(a){"use strict";function c(a,b){return a.line-b.line||a.ch-b.ch}function g(a,b,c,d){this.line=b,this.ch=c,this.cm=a,this.text=a.getLine(b),this.min=d?d.from:a.firstLine(),this.max=d?d.to-1:a.lastLine()}function h(a,c){var d=a.cm.getTokenTypeAt(b(a.line,c));return d&&/\btag\b/.test(d)}function i(a){return a.line>=a.max?void 0:(a.ch=0,a.text=a.cm.getLine(++a.line),!0)}function j(a){return a.line<=a.min?void 0:(a.text=a.cm.getLine(--a.line),a.ch=a.text.length,!0)}function k(a){for(;;){var b=a.text.indexOf(">",a.ch);if(-1==b){if(i(a))continue;return}{if(h(a,b+1)){var c=a.text.lastIndexOf("/",b),d=c>-1&&!/\S/.test(a.text.slice(c+1,b));return a.ch=b+1,d?"selfClose":"regular"}a.ch=b+1}}}function l(a){for(;;){var b=a.ch?a.text.lastIndexOf("<",a.ch-1):-1;if(-1==b){if(j(a))continue;return}if(h(a,b+1)){f.lastIndex=b,a.ch=b;var c=f.exec(a.text);if(c&&c.index==b)return c}else a.ch=b}}function m(a){for(;;){f.lastIndex=a.ch;var b=f.exec(a.text);if(!b){if(i(a))continue;return}{if(h(a,b.index+1))return a.ch=b.index+b[0].length,b;a.ch=b.index+1}}}function n(a){for(;;){var b=a.ch?a.text.lastIndexOf(">",a.ch-1):-1;if(-1==b){if(j(a))continue;return}{if(h(a,b+1)){var c=a.text.lastIndexOf("/",b),d=c>-1&&!/\S/.test(a.text.slice(c+1,b));return a.ch=b+1,d?"selfClose":"regular"}a.ch=b}}}function o(a,c){for(var d=[];;){var f,e=m(a),g=a.line,h=a.ch-(e?e[0].length:0);if(!e||!(f=k(a)))return;if("selfClose"!=f)if(e[1]){for(var i=d.length-1;i>=0;--i)if(d[i]==e[2]){d.length=i;break}if(0>i&&(!c||c==e[2]))return{tag:e[2],from:b(g,h),to:b(a.line,a.ch)}}else d.push(e[2])}}function p(a,c){for(var d=[];;){var e=n(a);if(!e)return;if("selfClose"!=e){var f=a.line,g=a.ch,h=l(a);if(!h)return;if(h[1])d.push(h[2]);else{for(var i=d.length-1;i>=0;--i)if(d[i]==h[2]){d.length=i;break}if(0>i&&(!c||c==h[2]))return{tag:h[2],from:b(a.line,a.ch),to:b(f,g)}}}else l(a)}}var b=a.Pos,d="A-Z_a-z\\u00C0-\\u00D6\\u00D8-\\u00F6\\u00F8-\\u02FF\\u0370-\\u037D\\u037F-\\u1FFF\\u200C-\\u200D\\u2070-\\u218F\\u2C00-\\u2FEF\\u3001-\\uD7FF\\uF900-\\uFDCF\\uFDF0-\\uFFFD",e=d+"-:.0-9\\u00B7\\u0300-\\u036F\\u203F-\\u2040",f=new RegExp("<(/?)(["+d+"]["+e+"]*)","g");a.registerHelper("fold","xml",function(a,c){for(var d=new g(a,c.line,0);;){var f,e=m(d);if(!e||d.line!=c.line||!(f=k(d)))return;if(!e[1]&&"selfClose"!=f){var c=b(d.line,d.ch),h=o(d,e[2]);return h&&{from:c,to:h.from}}}}),a.findMatchingTag=function(a,d,e){var f=new g(a,d.line,d.ch,e);if(-1!=f.text.indexOf(">")||-1!=f.text.indexOf("<")){var h=k(f),i=h&&b(f.line,f.ch),j=h&&l(f);if(h&&j&&!(c(f,d)>0)){var m={from:b(f.line,f.ch),to:i,tag:j[2]};return"selfClose"==h?{open:m,close:null,at:"open"}:j[1]?{open:p(f,j[2]),close:m,at:"close"}:(f=new g(a,i.line,i.ch,e),{open:m,close:o(f,j[2]),at:"open"})}}},a.findEnclosingTag=function(a,b,c){for(var d=new g(a,b.line,b.ch,c);;){var e=p(d);if(!e)break;var f=new g(a,b.line,b.ch,c),h=o(f,e.tag);if(h)return{open:e,close:h}}},a.scanForClosingTag=function(a,b,c,d){var e=new g(a,b.line,b.ch,d?{from:0,to:d}:null);return o(e,c)}}); \ No newline at end of file diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/jquery/combo/jquery.combo.css b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/jquery/combo/jquery.combo.css index 0432340acd..edf66f3e10 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/jquery/combo/jquery.combo.css +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/jquery/combo/jquery.combo.css @@ -86,3 +86,11 @@ div.combo-nifi-tooltip { background-color: #FFFFA3; color: #454545; } + +div.button-normal { + background: transparent url(../../../images/bgButton.png) repeat-x center center; +} + +div.button-over { + background: transparent url(../../../images/bgButtonOver.png) repeat-x center center; +} 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 11c3c0a821..1218f6c86b 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 @@ -29,8 +29,10 @@ nifi-web-api nifi-web-error nifi-web-docs + nifi-web-content-viewer nifi-web-ui nifi-jetty + nifi-web-content-access @@ -52,6 +54,12 @@ war 0.1.0-incubating-SNAPSHOT + + org.apache.nifi + nifi-web-content-viewer + war + 0.1.0-incubating-SNAPSHOT + org.apache.nifi nifi-web-ui diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/pom.xml b/nifi/nifi-nar-bundles/nifi-framework-bundle/pom.xml index 074821a7fb..28fb24c837 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/pom.xml +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/pom.xml @@ -63,6 +63,11 @@ nifi-client-dto 0.1.0-incubating-SNAPSHOT + + org.apache.nifi + nifi-web-content-access + 0.1.0-incubating-SNAPSHOT + org.apache.nifi nifi-security diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-content-viewer/pom.xml b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-content-viewer/pom.xml new file mode 100644 index 0000000000..1b1317845a --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-content-viewer/pom.xml @@ -0,0 +1,71 @@ + + + 4.0.0 + + org.apache.nifi + nifi-standard-bundle + 0.1.0-incubating-SNAPSHOT + + org.apache.nifi + nifi-standard-content-viewer + war + + + org.apache.commons + commons-lang3 + provided + + + commons-codec + commons-codec + provided + + + commons-io + commons-io + provided + + + org.codehaus.jackson + jackson-mapper-asl + 1.9.13 + + + org.apache.nifi + nifi-api + + + javax.servlet.jsp + javax.servlet.jsp-api + provided + + + javax.el + javax.el-api + provided + + + javax.servlet.jsp.jstl + javax.servlet.jsp.jstl-api + provided + + + javax.servlet + javax.servlet-api + provided + + + diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-content-viewer/src/main/java/org/apache/nifi/web/StandardContentViewerController.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-content-viewer/src/main/java/org/apache/nifi/web/StandardContentViewerController.java new file mode 100644 index 0000000000..6ce315edb6 --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-content-viewer/src/main/java/org/apache/nifi/web/StandardContentViewerController.java @@ -0,0 +1,103 @@ +/* + * 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.web; + +import java.io.IOException; +import java.io.PrintWriter; +import java.io.StringWriter; + +import javax.servlet.ServletException; +import javax.servlet.annotation.WebServlet; +import javax.servlet.http.HttpServlet; +import javax.servlet.http.HttpServletRequest; +import javax.servlet.http.HttpServletResponse; +import javax.xml.transform.OutputKeys; +import javax.xml.transform.Transformer; +import javax.xml.transform.TransformerException; +import javax.xml.transform.TransformerFactory; +import javax.xml.transform.TransformerFactoryConfigurationError; +import javax.xml.transform.stream.StreamResult; +import javax.xml.transform.stream.StreamSource; +import org.apache.nifi.web.ViewableContent.DisplayMode; +import org.codehaus.jackson.map.ObjectMapper; + +/** + * + */ +@WebServlet(name = "StandardContentViewer", urlPatterns = {"/view-content"}) +public class StandardContentViewerController extends HttpServlet { + + /** + * + * @param request servlet request + * @param response servlet response + * @throws ServletException if a servlet-specific error occurs + * @throws IOException if an I/O error occurs + */ + @Override + protected void doGet(HttpServletRequest request, HttpServletResponse response) throws ServletException, IOException { + final ViewableContent content = (ViewableContent) request.getAttribute(ViewableContent.CONTENT_REQUEST_ATTRIBUTE); + + // handle json/xml + if ("application/json".equals(content.getContentType()) || "application/xml".equals(content.getContentType()) || "text/plain".equals(content.getContentType())) { + final String formatted; + + // leave the content alone if specified + if (DisplayMode.Original.equals(content.getDisplayMode())) { + formatted = content.getContent(); + } else { + if ("application/json".equals(content.getContentType())) { + // format json + final ObjectMapper mapper = new ObjectMapper(); + final Object objectJson = mapper.readValue(content.getContent(), Object.class); + formatted = mapper.writerWithDefaultPrettyPrinter().writeValueAsString(objectJson); + } else if ("application/xml".equals(content.getContentType())) { + // format xml + final StringWriter writer = new StringWriter(); + + try { + final StreamSource source = new StreamSource(content.getContentStream()); + final StreamResult result = new StreamResult(writer); + + final TransformerFactory transformFactory = TransformerFactory.newInstance(); + final Transformer transformer = transformFactory.newTransformer(); + transformer.setOutputProperty("{http://xml.apache.org/xslt}indent-amount", "2"); + transformer.setOutputProperty(OutputKeys.INDENT, "yes"); + + transformer.transform(source, result); + } catch (final TransformerFactoryConfigurationError | TransformerException te) { + throw new IOException("Unable to transform content as XML: " + te, te); + } + + // get the transformed xml + formatted = writer.toString(); + } else { + // leave plain text alone when formatting + formatted = content.getContent(); + } + } + + // defer to the jsp + request.setAttribute("mode", content.getContentType()); + request.setAttribute("content", formatted); + request.getRequestDispatcher("/WEB-INF/jsp/codemirror.jsp").include(request, response); + } else { + final PrintWriter out = response.getWriter(); + out.println("Unexpected content type: " + content.getContentType()); + } + } +} diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-content-viewer/src/main/resources/META-INF/NOTICE b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-content-viewer/src/main/resources/META-INF/NOTICE new file mode 100644 index 0000000000..d91a95247a --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-content-viewer/src/main/resources/META-INF/NOTICE @@ -0,0 +1,19 @@ +nifi-web-docs +Copyright 2014-2015 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + +=========================================== +Apache Software License v2 +=========================================== + +The following binary components are provided under the Apache Software License v2 + + (ASLv2) Apache Commons Lang + The following NOTICE information applies: + Apache Commons Lang + Copyright 2001-2014 The Apache Software Foundation + + This product includes software from the Spring Framework, + under the Apache License 2.0 (see: StringUtils.containsWhitespace()) diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-content-viewer/src/main/webapp/META-INF/nifi-content-viewer b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-content-viewer/src/main/webapp/META-INF/nifi-content-viewer new file mode 100644 index 0000000000..cbf30b7496 --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-content-viewer/src/main/webapp/META-INF/nifi-content-viewer @@ -0,0 +1,3 @@ +application/xml +application/json +text/plain \ No newline at end of file diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-content-viewer/src/main/webapp/WEB-INF/jsp/codemirror.jsp b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-content-viewer/src/main/webapp/WEB-INF/jsp/codemirror.jsp new file mode 100644 index 0000000000..65ad826532 --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-content-viewer/src/main/webapp/WEB-INF/jsp/codemirror.jsp @@ -0,0 +1,47 @@ +<%-- + 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. +--%> +<%@ page contentType="text/html" pageEncoding="UTF-8" session="false" %> + + + + + + + + \ No newline at end of file diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-content-viewer/src/main/webapp/WEB-INF/web.xml b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-content-viewer/src/main/webapp/WEB-INF/web.xml new file mode 100644 index 0000000000..8f59916cab --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-content-viewer/src/main/webapp/WEB-INF/web.xml @@ -0,0 +1,29 @@ + + + + nifi-standard-content-viewer + + StandardContentViewer + org.apache.nifi.web.StandardContentViewerController + + + StandardContentViewer + /view-content + + + view-content + + diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-content-viewer/src/main/webapp/css/main.css b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-content-viewer/src/main/webapp/css/main.css new file mode 100644 index 0000000000..dad3120d5c --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-content-viewer/src/main/webapp/css/main.css @@ -0,0 +1,20 @@ +/* + * 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. + */ +* { + margin: 0; + padding: 0; +} \ No newline at end of file diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-nar/pom.xml b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-nar/pom.xml index 8830a15dad..15b285da4d 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-nar/pom.xml +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-nar/pom.xml @@ -40,5 +40,10 @@ org.apache.nifi nifi-standard-reporting-tasks + + org.apache.nifi + nifi-standard-content-viewer + war + diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/pom.xml b/nifi/nifi-nar-bundles/nifi-standard-bundle/pom.xml index 1701c85081..6c00f1f4a1 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/pom.xml +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/pom.xml @@ -1,3 +1,4 @@ + + + + MyProcessor + + + + +

      This is a place to provide additional text, images and references that best describe how your processor is used +

      + + + diff --git a/nifi/nifi-maven-archetypes/nifi-processor-bundle-archetype/src/main/resources/archetype-resources/nifi-__artifactBaseName__-processors/src/main/resources/docs/__package__.MyProcessor/index.html b/nifi/nifi-maven-archetypes/nifi-processor-bundle-archetype/src/main/resources/archetype-resources/nifi-__artifactBaseName__-processors/src/main/resources/docs/__package__.MyProcessor/index.html deleted file mode 100644 index 34e494fa5c..0000000000 --- a/nifi/nifi-maven-archetypes/nifi-processor-bundle-archetype/src/main/resources/archetype-resources/nifi-__artifactBaseName__-processors/src/main/resources/docs/__package__.MyProcessor/index.html +++ /dev/null @@ -1,96 +0,0 @@ - - - - - - GetFile - - - - - -

      Description:

      - -

      Replace with a description.

      - -

      - Uses Attributes: -

      - - - - - - - - - - - - - -
      Attribute NameDescription
      - -

      - Modifies Attributes: -

      - - - - - - - - - - - - - -
      Attribute NameDescription
      - -

      - 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.

      -
        -
      • My Property -
          -
        • An example property.
        • -
        • Default value: no default
        • -
        • Supports expression language: false
        • -
        -
      • -
      - -

      - Relationships: -

      -
        -
      • my_relationship -
          -
        • An example relationship.
        • -
        -
      • -
      - - - From efeba19cb6c72585ecaacbbb6662a274fab8f477 Mon Sep 17 00:00:00 2001 From: danbress Date: Thu, 19 Mar 2015 08:32:42 -0400 Subject: [PATCH 066/116] NIFI-441 Defaulting nifi version to 0.1.0-incubating-SNAPSHOT --- .../src/main/resources/META-INF/maven/archetype-metadata.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/nifi/nifi-maven-archetypes/nifi-processor-bundle-archetype/src/main/resources/META-INF/maven/archetype-metadata.xml b/nifi/nifi-maven-archetypes/nifi-processor-bundle-archetype/src/main/resources/META-INF/maven/archetype-metadata.xml index 085bd99834..979916c356 100644 --- a/nifi/nifi-maven-archetypes/nifi-processor-bundle-archetype/src/main/resources/META-INF/maven/archetype-metadata.xml +++ b/nifi/nifi-maven-archetypes/nifi-processor-bundle-archetype/src/main/resources/META-INF/maven/archetype-metadata.xml @@ -17,7 +17,7 @@ - 0.0.1-incubating-SNAPSHOT + 0.1.0-incubating-SNAPSHOT ${groupId}.processors.${artifactBaseName} From f63e09b9b8480d3bbe78beee4b12cd5cb3ed8979 Mon Sep 17 00:00:00 2001 From: danbress Date: Thu, 19 Mar 2015 08:37:27 -0400 Subject: [PATCH 067/116] NIFI-441 - Adding SeeAlso, ReadsAttributes and WritesAttributes --- .../src/main/java/MyProcessor.java | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/nifi/nifi-maven-archetypes/nifi-processor-bundle-archetype/src/main/resources/archetype-resources/nifi-__artifactBaseName__-processors/src/main/java/MyProcessor.java b/nifi/nifi-maven-archetypes/nifi-processor-bundle-archetype/src/main/resources/archetype-resources/nifi-__artifactBaseName__-processors/src/main/java/MyProcessor.java index 7d29b5e466..8d9d261526 100644 --- a/nifi/nifi-maven-archetypes/nifi-processor-bundle-archetype/src/main/resources/archetype-resources/nifi-__artifactBaseName__-processors/src/main/java/MyProcessor.java +++ b/nifi/nifi-maven-archetypes/nifi-processor-bundle-archetype/src/main/resources/archetype-resources/nifi-__artifactBaseName__-processors/src/main/java/MyProcessor.java @@ -22,7 +22,12 @@ import org.apache.nifi.flowfile.FlowFile; import org.apache.nifi.processor.*; import org.apache.nifi.annotation.lifecycle.OnScheduled; import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.documentation.ReadsAttribute; +import org.apache.nifi.annotation.documentation.ReadsAttributes; +import org.apache.nifi.annotation.documentation.SeeAlso; import org.apache.nifi.annotation.documentation.Tags; +import org.apache.nifi.annotation.documentation.WritesAttribute; +import org.apache.nifi.annotation.documentation.WritesAttributes; import org.apache.nifi.processor.exception.ProcessException; import org.apache.nifi.processor.util.StandardValidators; @@ -30,6 +35,9 @@ import java.util.*; @Tags({"example"}) @CapabilityDescription("Provide a description") +@SeeAlso({}) +@ReadsAttributes({@ReadsAttribute(attribute="", description="")}) +@WritesAttributes({@WritesAttribute(attribute="", description="")}) public class MyProcessor extends AbstractProcessor { public static final PropertyDescriptor MY_PROPERTY = new PropertyDescriptor From 0f6fb7ecac8fc11a11d947db4fbf310fa1784e34 Mon Sep 17 00:00:00 2001 From: danbress Date: Thu, 19 Mar 2015 08:37:44 -0400 Subject: [PATCH 068/116] NIFI-441 updating additionalDetails.html --- .../docs/__package__.MyProcessor/additionalDetails.html | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/nifi/nifi-maven-archetypes/nifi-processor-bundle-archetype/src/main/resources/archetype-resources/nifi-__artifactBaseName__-processors/src/main/resources/docs/__package__.MyProcessor/additionalDetails.html b/nifi/nifi-maven-archetypes/nifi-processor-bundle-archetype/src/main/resources/archetype-resources/nifi-__artifactBaseName__-processors/src/main/resources/docs/__package__.MyProcessor/additionalDetails.html index 235cffa586..533754371c 100644 --- a/nifi/nifi-maven-archetypes/nifi-processor-bundle-archetype/src/main/resources/archetype-resources/nifi-__artifactBaseName__-processors/src/main/resources/docs/__package__.MyProcessor/additionalDetails.html +++ b/nifi/nifi-maven-archetypes/nifi-processor-bundle-archetype/src/main/resources/archetype-resources/nifi-__artifactBaseName__-processors/src/main/resources/docs/__package__.MyProcessor/additionalDetails.html @@ -21,7 +21,7 @@ -

      This is a place to provide additional text, images and references that best describe how your processor is used +

      This is a place to provide additional text, images and references that best describe how to use your processor. If this level of detail is not necessary, you can remove this file.

      From 3359891c660020cf583a92e738f9ba1b75f21aae Mon Sep 17 00:00:00 2001 From: danbress Date: Mon, 23 Mar 2015 08:34:41 -0400 Subject: [PATCH 069/116] NIFI-441 incorporating changes from moving annotations --- .../src/main/java/MyProcessor.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/nifi/nifi-maven-archetypes/nifi-processor-bundle-archetype/src/main/resources/archetype-resources/nifi-__artifactBaseName__-processors/src/main/java/MyProcessor.java b/nifi/nifi-maven-archetypes/nifi-processor-bundle-archetype/src/main/resources/archetype-resources/nifi-__artifactBaseName__-processors/src/main/java/MyProcessor.java index 8d9d261526..7b70dcab7f 100644 --- a/nifi/nifi-maven-archetypes/nifi-processor-bundle-archetype/src/main/resources/archetype-resources/nifi-__artifactBaseName__-processors/src/main/java/MyProcessor.java +++ b/nifi/nifi-maven-archetypes/nifi-processor-bundle-archetype/src/main/resources/archetype-resources/nifi-__artifactBaseName__-processors/src/main/java/MyProcessor.java @@ -20,14 +20,14 @@ import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.components.PropertyValue; import org.apache.nifi.flowfile.FlowFile; import org.apache.nifi.processor.*; +import org.apache.nifi.annotation.behavior.ReadsAttribute; +import org.apache.nifi.annotation.behavior.ReadsAttributes; +import org.apache.nifi.annotation.behavior.WritesAttribute; +import org.apache.nifi.annotation.behavior.WritesAttributes; import org.apache.nifi.annotation.lifecycle.OnScheduled; import org.apache.nifi.annotation.documentation.CapabilityDescription; -import org.apache.nifi.annotation.documentation.ReadsAttribute; -import org.apache.nifi.annotation.documentation.ReadsAttributes; import org.apache.nifi.annotation.documentation.SeeAlso; import org.apache.nifi.annotation.documentation.Tags; -import org.apache.nifi.annotation.documentation.WritesAttribute; -import org.apache.nifi.annotation.documentation.WritesAttributes; import org.apache.nifi.processor.exception.ProcessException; import org.apache.nifi.processor.util.StandardValidators; From e78aeddb6937d5ed54f73d1f6dcf11547181e374 Mon Sep 17 00:00:00 2001 From: danbress Date: Mon, 23 Mar 2015 09:59:50 -0400 Subject: [PATCH 070/116] NIFI-441 removing additionalDetails.html --- .../additionalDetails.html | 28 ------------------- 1 file changed, 28 deletions(-) delete mode 100644 nifi/nifi-maven-archetypes/nifi-processor-bundle-archetype/src/main/resources/archetype-resources/nifi-__artifactBaseName__-processors/src/main/resources/docs/__package__.MyProcessor/additionalDetails.html diff --git a/nifi/nifi-maven-archetypes/nifi-processor-bundle-archetype/src/main/resources/archetype-resources/nifi-__artifactBaseName__-processors/src/main/resources/docs/__package__.MyProcessor/additionalDetails.html b/nifi/nifi-maven-archetypes/nifi-processor-bundle-archetype/src/main/resources/archetype-resources/nifi-__artifactBaseName__-processors/src/main/resources/docs/__package__.MyProcessor/additionalDetails.html deleted file mode 100644 index 533754371c..0000000000 --- a/nifi/nifi-maven-archetypes/nifi-processor-bundle-archetype/src/main/resources/archetype-resources/nifi-__artifactBaseName__-processors/src/main/resources/docs/__package__.MyProcessor/additionalDetails.html +++ /dev/null @@ -1,28 +0,0 @@ - - - - - - MyProcessor - - - - -

      This is a place to provide additional text, images and references that best describe how to use your processor. If this level of detail is not necessary, you can remove this file. -

      - - - From b97396c0fabe5a13043e6243c2e578e527b4f0c5 Mon Sep 17 00:00:00 2001 From: Brian Ghigiarelli Date: Fri, 20 Mar 2015 16:37:23 -0400 Subject: [PATCH 071/116] Adding TLS to PutEmail and several other PropertyDescriptors that can be driven by EL for full control of outgoing emails. Also switched from Sun SMTPTransport to Java Transport for sending the email message Signed-off-by: Mark Payne --- .../nifi/processors/standard/PutEmail.java | 149 +++++++++++++++--- .../processors/standard/TestPutEmail.java | 40 ++++- 2 files changed, 167 insertions(+), 22 deletions(-) diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PutEmail.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PutEmail.java index d83b100420..f6320005f1 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PutEmail.java +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PutEmail.java @@ -22,18 +22,22 @@ import java.util.ArrayList; import java.util.Collection; import java.util.Collections; import java.util.Date; +import java.util.HashMap; import java.util.HashSet; import java.util.List; +import java.util.Map; import java.util.Map.Entry; import java.util.Properties; import java.util.Set; import javax.activation.DataHandler; +import javax.mail.Authenticator; import javax.mail.Message; import javax.mail.Message.RecipientType; import javax.mail.MessagingException; +import javax.mail.PasswordAuthentication; import javax.mail.Session; -import javax.mail.URLName; +import javax.mail.Transport; import javax.mail.internet.AddressException; import javax.mail.internet.InternetAddress; import javax.mail.internet.MimeBodyPart; @@ -61,8 +65,6 @@ import org.apache.nifi.processor.exception.ProcessException; import org.apache.nifi.processor.io.InputStreamCallback; import org.apache.nifi.processor.util.StandardValidators; -import com.sun.mail.smtp.SMTPTransport; - @SupportsBatching @Tags({"email", "put", "notify", "smtp"}) @CapabilityDescription("Sends an e-mail to configured recipients for each incoming FlowFile") @@ -72,6 +74,7 @@ public class PutEmail extends AbstractProcessor { .name("SMTP Hostname") .description("The hostname of the SMTP host") .required(true) + .expressionLanguageSupported(true) .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) .build(); public static final PropertyDescriptor SMTP_PORT = new PropertyDescriptor.Builder() @@ -79,21 +82,56 @@ public class PutEmail extends AbstractProcessor { .description("The Port used for SMTP communications") .required(true) .defaultValue("25") + .expressionLanguageSupported(true) .addValidator(StandardValidators.PORT_VALIDATOR) .build(); public static final PropertyDescriptor SMTP_USERNAME = new PropertyDescriptor.Builder() .name("SMTP Username") .description("Username for the SMTP account") + .expressionLanguageSupported(true) .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) .required(false) .build(); public static final PropertyDescriptor SMTP_PASSWORD = new PropertyDescriptor.Builder() .name("SMTP Password") .description("Password for the SMTP account") + .expressionLanguageSupported(true) .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) .required(false) .sensitive(true) .build(); + public static final PropertyDescriptor SMTP_AUTH = new PropertyDescriptor.Builder() + .name("SMTP Auth") + .description("Flag indicating whether authentication should be used") + .required(true) + .expressionLanguageSupported(true) + .addValidator(StandardValidators.BOOLEAN_VALIDATOR) + .defaultValue("true") + .build(); + public static final PropertyDescriptor SMTP_TLS = new PropertyDescriptor.Builder() + .name("SMTP TLS") + .description("Flag indicating whether TLS should be enabled") + .required(true) + .expressionLanguageSupported(true) + .addValidator(StandardValidators.BOOLEAN_VALIDATOR) + .defaultValue("false") + .build(); + public static final PropertyDescriptor SMTP_SOCKET_FACTORY = new PropertyDescriptor.Builder() + .name("SMTP Socket Factory") + .description("Socket Factory to use for SMTP Connection") + .required(true) + .expressionLanguageSupported(true) + .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) + .defaultValue("javax.net.ssl.SSLSocketFactory") + .build(); + public static final PropertyDescriptor HEADER_XMAILER = new PropertyDescriptor.Builder() + .name("SMTP X-Mailer Header") + .description("X-Mailer used in the header of the outgoing email") + .required(true) + .expressionLanguageSupported(true) + .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) + .defaultValue("NiFi") + .build(); public static final PropertyDescriptor FROM = new PropertyDescriptor.Builder() .name("From") .description("Specifies the Email address to use as the sender") @@ -152,12 +190,27 @@ public class PutEmail extends AbstractProcessor { .allowableValues("true", "false") .defaultValue("false") .build(); - + public static final Relationship REL_SUCCESS = new Relationship.Builder().name("success").description("FlowFiles that are successfully sent will be routed to this relationship").build(); public static final Relationship REL_FAILURE = new Relationship.Builder().name("failure").description("FlowFiles that fail to send will be routed to this relationship").build(); private List properties; private Set relationships; + + /** + * Mapping of the mail properties to the NiFi PropertyDescriptors that will be evaluated at runtime + */ + private static Map propertyToContext = new HashMap(); + static { + propertyToContext.put("mail.smtp.host", SMTP_HOSTNAME); + propertyToContext.put("mail.smtp.port", SMTP_PORT); + propertyToContext.put("mail.smtp.socketFactory.port", SMTP_PORT); + propertyToContext.put("mail.smtp.socketFactory.class", SMTP_SOCKET_FACTORY); + propertyToContext.put("mail.smtp.auth", SMTP_AUTH); + propertyToContext.put("mail.smtp.starttls.enable", SMTP_TLS); + propertyToContext.put("mail.smtp.user", SMTP_USERNAME); + propertyToContext.put("mail.smtp.password", SMTP_PASSWORD); + } @Override protected void init(final ProcessorInitializationContext context) { @@ -166,6 +219,10 @@ public class PutEmail extends AbstractProcessor { properties.add(SMTP_PORT); properties.add(SMTP_USERNAME); properties.add(SMTP_PASSWORD); + properties.add(SMTP_AUTH); + properties.add(SMTP_TLS); + properties.add(SMTP_SOCKET_FACTORY); + properties.add(HEADER_XMAILER); properties.add(FROM); properties.add(TO); properties.add(CC); @@ -214,9 +271,10 @@ public class PutEmail extends AbstractProcessor { return; } - final Properties properties = new Properties(); - properties.setProperty("smtp.mail.host", context.getProperty(SMTP_HOSTNAME).getValue()); - final Session mailSession = Session.getInstance(properties); + final Properties properties = this.getMailPropertiesFromFlowFile(context, flowFile); + + final Session mailSession = this.createMailSession(properties); + final Message message = new MimeMessage(mailSession); final ProcessorLog logger = getLogger(); @@ -232,7 +290,7 @@ public class PutEmail extends AbstractProcessor { final InternetAddress[] bccAddresses = toInetAddresses(context.getProperty(BCC).evaluateAttributeExpressions(flowFile).getValue()); message.setRecipients(RecipientType.BCC, bccAddresses); - message.setHeader("X-Mailer", "NiFi"); + message.setHeader("X-Mailer", context.getProperty(HEADER_XMAILER).evaluateAttributeExpressions(flowFile).getValue()); message.setSubject(context.getProperty(SUBJECT).evaluateAttributeExpressions(flowFile).getValue()); String messageText = context.getProperty(MESSAGE).evaluateAttributeExpressions(flowFile).getValue(); @@ -264,18 +322,8 @@ public class PutEmail extends AbstractProcessor { multipart.addBodyPart(mimeFile); message.setContent(multipart); } - - final String smtpHost = context.getProperty(SMTP_HOSTNAME).getValue(); - final SMTPTransport transport = new SMTPTransport(mailSession, new URLName(smtpHost)); - try { - final int smtpPort = context.getProperty(SMTP_PORT).asInteger(); - final String smtpUsername = context.getProperty(SMTP_USERNAME).getValue(); - final String smtpPassword = context.getProperty(SMTP_PASSWORD).getValue(); - transport.connect(smtpHost, smtpPort, smtpUsername, smtpPassword); - transport.sendMessage(message, message.getAllRecipients()); - } finally { - transport.close(); - } + + Transport.send(message); session.getProvenanceReporter().send(flowFile, "mailto:" + message.getAllRecipients()[0].toString()); session.transfer(flowFile, REL_SUCCESS); @@ -287,7 +335,66 @@ public class PutEmail extends AbstractProcessor { } } - public static final String BODY_SEPARATOR = "\n\n--------------------------------------------------\n"; + /** + * Based on the input properties, determine whether an authenticate or unauthenticated session + * should be used. If authenticated, creates a Password Authenticator for use in sending the email. + * + * @param properties + * @return + */ + private Session createMailSession(final Properties properties) { + String authValue = properties.getProperty("mail.smtp.auth"); + Boolean auth = Boolean.valueOf(authValue); + + /* + * Conditionally create a password authenticator if the 'auth' parameter is set. + */ + final Session mailSession = auth ? Session.getInstance(properties, new Authenticator() { + @Override + public PasswordAuthentication getPasswordAuthentication() { + String username = properties.getProperty("mail.smtp.user"), + password = properties.getProperty("mail.smtp.password"); + return new PasswordAuthentication(username, password); + } + }) : Session.getInstance(properties); // without auth + return mailSession; + } + + /** + * Uses the mapping of javax.mail properties to NiFi PropertyDescriptors to build + * the required Properties object to be used for sending this email + * + * @param context + * @param flowFile + * @return + */ + private Properties getMailPropertiesFromFlowFile(final ProcessContext context, final FlowFile flowFile) { + + final Properties properties = new Properties(); + + final ProcessorLog logger = this.getLogger(); + + for(Entry entry : propertyToContext.entrySet()) { + + // Evaluate the property descriptor against the flow file + String flowFileValue = context.getProperty(entry.getValue()).evaluateAttributeExpressions(flowFile).getValue(); + + String property = entry.getKey(); + + logger.debug("Evaluated Mail Property: {} with Value: {}", new Object[]{property, flowFileValue}); + + // Nullable values are not allowed, so filter out + if(null != flowFileValue) { + properties.setProperty(property, flowFileValue); + } + + } + + return properties; + + } + + public static final String BODY_SEPARATOR = "\n\n--------------------------------------------------\n"; private static String formatAttributes(final FlowFile flowFile, final String messagePrepend) { StringBuilder message = new StringBuilder(messagePrepend); diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestPutEmail.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestPutEmail.java index b737ed6173..b0ce47747e 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestPutEmail.java +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestPutEmail.java @@ -19,14 +19,19 @@ package org.apache.nifi.processors.standard; import java.util.HashMap; import java.util.Map; +import org.apache.nifi.flowfile.FlowFile; +import org.apache.nifi.processor.ProcessContext; +import org.apache.nifi.processor.ProcessSession; import org.apache.nifi.util.TestRunner; import org.apache.nifi.util.TestRunners; import org.junit.Test; +import static org.junit.Assert.*; + public class TestPutEmail { @Test - public void testHotNotFound() { + public void testHostNotFound() { // verifies that files are routed to failure when the SMTP host doesn't exist final TestRunner runner = TestRunners.newTestRunner(new PutEmail()); runner.setProperty(PutEmail.SMTP_HOSTNAME, "host-doesnt-exist123"); @@ -42,4 +47,37 @@ public class TestPutEmail { runner.assertQueueEmpty(); runner.assertAllFlowFilesTransferred(PutEmail.REL_FAILURE); } + + @Test + public void testEmailPropertyFormatters() { + // verifies that files are routed to failure when the SMTP host doesn't exist + final TestRunner runner = TestRunners.newTestRunner(new PutEmail()); + runner.setProperty(PutEmail.HEADER_XMAILER, "TestingNiFi"); + runner.setProperty(PutEmail.SMTP_HOSTNAME, "smtp-host"); + runner.setProperty(PutEmail.SMTP_SOCKET_FACTORY, "${dynamicSocketFactory}"); + runner.setProperty(PutEmail.HEADER_XMAILER, "TestingNiFi"); + runner.setProperty(PutEmail.FROM, "test@apache.org"); + runner.setProperty(PutEmail.MESSAGE, "Message Body"); + runner.setProperty(PutEmail.TO, "recipient@apache.org"); + + ProcessSession session = runner.getProcessSessionFactory().createSession(); + FlowFile ff = session.create(); + ff = session.putAttribute(ff, "dynamicSocketFactory", "testingSocketFactory"); + ProcessContext context = runner.getProcessContext(); + + String xmailer = context.getProperty(PutEmail.HEADER_XMAILER).evaluateAttributeExpressions(ff).getValue(); + assertEquals("X-Mailer Header", "TestingNiFi", xmailer); + + String socketFactory = context.getProperty(PutEmail.SMTP_SOCKET_FACTORY).evaluateAttributeExpressions(ff).getValue(); + assertEquals("Socket Factory", "testingSocketFactory", socketFactory); + + final Map attributes = new HashMap<>(); + runner.enqueue("Some Text".getBytes(), attributes); + + runner.run(); + + runner.assertQueueEmpty(); + runner.assertAllFlowFilesTransferred(PutEmail.REL_FAILURE); + } + } From 0c5abce5f95876320586453d1391fe2588371ec3 Mon Sep 17 00:00:00 2001 From: danbress Date: Mon, 23 Mar 2015 09:14:09 -0400 Subject: [PATCH 072/116] NIFI-434 putting ids on html element to facilitate CSS --- .../html/HtmlDocumentationWriter.java | 30 +++++++++++++++++-- .../HtmlProcessorDocumentationWriter.java | 4 +++ 2 files changed, 31 insertions(+), 3 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 b1f426ae6a..0da5bedd98 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 @@ -302,12 +302,13 @@ public class HtmlDocumentationWriter implements DocumentationWriter { List properties = configurableComponent.getPropertyDescriptors(); if (properties.size() > 0) { xmlStreamWriter.writeStartElement("table"); + xmlStreamWriter.writeAttribute("id", "properties"); // write the header row xmlStreamWriter.writeStartElement("tr"); writeSimpleElement(xmlStreamWriter, "th", "Name"); writeSimpleElement(xmlStreamWriter, "th", "Default Value"); - writeSimpleElement(xmlStreamWriter, "th", "Valid Values"); + writeSimpleElement(xmlStreamWriter, "th", "Allowable Values"); writeSimpleElement(xmlStreamWriter, "th", "Description"); xmlStreamWriter.writeEndElement(); @@ -315,6 +316,7 @@ public class HtmlDocumentationWriter implements DocumentationWriter { for (PropertyDescriptor property : properties) { xmlStreamWriter.writeStartElement("tr"); xmlStreamWriter.writeStartElement("td"); + xmlStreamWriter.writeAttribute("id", "name"); if (property.isRequired()) { writeSimpleElement(xmlStreamWriter, "strong", property.getDisplayName()); } else { @@ -322,11 +324,13 @@ public class HtmlDocumentationWriter implements DocumentationWriter { } xmlStreamWriter.writeEndElement(); - writeSimpleElement(xmlStreamWriter, "td", property.getDefaultValue()); + writeSimpleElement(xmlStreamWriter, "td", property.getDefaultValue(), false, "default-value"); xmlStreamWriter.writeStartElement("td"); + xmlStreamWriter.writeAttribute("id", "allowable-values"); writeValidValues(xmlStreamWriter, property); xmlStreamWriter.writeEndElement(); xmlStreamWriter.writeStartElement("td"); + xmlStreamWriter.writeAttribute("id", "description"); if (property.getDescription() != null && property.getDescription().trim().length() > 0) { xmlStreamWriter.writeCharacters(property.getDescription()); } else { @@ -372,6 +376,7 @@ public class HtmlDocumentationWriter implements DocumentationWriter { xmlStreamWriter .writeCharacters("Dynamic Properties allow the user to specify both the name and value of a property."); xmlStreamWriter.writeStartElement("table"); + xmlStreamWriter.writeAttribute("id", "dynamic-properties"); xmlStreamWriter.writeStartElement("tr"); writeSimpleElement(xmlStreamWriter, "th", "Name"); writeSimpleElement(xmlStreamWriter, "th", "Value"); @@ -489,7 +494,26 @@ public class HtmlDocumentationWriter implements DocumentationWriter { */ protected final static void writeSimpleElement(final XMLStreamWriter writer, final String elementName, final String characters, boolean strong) throws XMLStreamException { - writer.writeStartElement(elementName); + writeSimpleElement(writer, elementName, characters, strong, null); + } + + /** + * Writes a begin element, an id attribute(if specified), then text, then end element for + * element of the users choosing. Example: <p id="p-id">text</p> + * + * @param writer the stream writer to use + * @param elementName the name of the element + * @param characters the text of the element + * @param strong whether to bold the text of the element or not + * @param id the id of the element. specifying null will cause no element to be written. + * @throws XMLStreamException + */ + protected final static void writeSimpleElement(final XMLStreamWriter writer, final String elementName, + final String characters, boolean strong, String id) throws XMLStreamException { + writer.writeStartElement(elementName); + if (id != null) { + writer.writeAttribute("id", id); + } if (strong) { writer.writeStartElement("strong"); } 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 f210c7a610..bc355f7712 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 @@ -83,6 +83,7 @@ public class HtmlProcessorDocumentationWriter extends HtmlDocumentationWriter { writeSimpleElement(xmlStreamWriter, "h3", "Reads Attributes: "); if (attributesRead.size() > 0) { xmlStreamWriter.writeStartElement("table"); + xmlStreamWriter.writeAttribute("id", "reads-attributes"); xmlStreamWriter.writeStartElement("tr"); writeSimpleElement(xmlStreamWriter, "th", "Name"); writeSimpleElement(xmlStreamWriter, "th", "Description"); @@ -120,6 +121,7 @@ public class HtmlProcessorDocumentationWriter extends HtmlDocumentationWriter { writeSimpleElement(xmlStreamWriter, "h3", "Writes Attributes: "); if (attributesRead.size() > 0) { xmlStreamWriter.writeStartElement("table"); + xmlStreamWriter.writeAttribute("id", "writes-attributes"); xmlStreamWriter.writeStartElement("tr"); writeSimpleElement(xmlStreamWriter, "th", "Name"); writeSimpleElement(xmlStreamWriter, "th", "Description"); @@ -203,6 +205,7 @@ public class HtmlProcessorDocumentationWriter extends HtmlDocumentationWriter { if (processor.getRelationships().size() > 0) { xmlStreamWriter.writeStartElement("table"); + xmlStreamWriter.writeAttribute("id", "relationships"); xmlStreamWriter.writeStartElement("tr"); writeSimpleElement(xmlStreamWriter, "th", "Name"); writeSimpleElement(xmlStreamWriter, "th", "Description"); @@ -236,6 +239,7 @@ public class HtmlProcessorDocumentationWriter extends HtmlDocumentationWriter { xmlStreamWriter.writeStartElement("p"); xmlStreamWriter.writeCharacters("A Dynamic Relationship may be created based on how the user configures the Processor."); xmlStreamWriter.writeStartElement("table"); + xmlStreamWriter.writeAttribute("id", "dynamic-relationships"); xmlStreamWriter.writeStartElement("tr"); writeSimpleElement(xmlStreamWriter, "th", "Name"); writeSimpleElement(xmlStreamWriter, "th", "Description"); From 2f09e6ec5a1b1b4f652d46a804962a048adaceb9 Mon Sep 17 00:00:00 2001 From: danbress Date: Mon, 23 Mar 2015 09:14:29 -0400 Subject: [PATCH 073/116] NIFI-434 adjusting CSS to not break on spaces for the first three columns of a property table --- .../nifi-web-docs/src/main/webapp/css/component-usage.css | 4 ++++ 1 file changed, 4 insertions(+) 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 45c1e267ac..6aea18b9e5 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 @@ -113,6 +113,10 @@ table tr:last-child td:last-child { border-bottom-right-radius:3px; } +td#allowable-values, td#default-value, td#name { + white-space:nowrap; +} + /* links */ a, a:link, a:visited { From 10f724753c6e819a11a295baba0d79e292617beb Mon Sep 17 00:00:00 2001 From: danbress Date: Mon, 23 Mar 2015 09:39:08 -0400 Subject: [PATCH 074/116] NIFI-434 adding additional ids to dynamic property tds --- .../nifi/documentation/html/HtmlDocumentationWriter.java | 4 ++-- 1 file changed, 2 insertions(+), 2 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 0da5bedd98..34b1327513 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 @@ -384,8 +384,8 @@ public class HtmlDocumentationWriter implements DocumentationWriter { xmlStreamWriter.writeEndElement(); for (final DynamicProperty dynamicProperty : dynamicProperties) { xmlStreamWriter.writeStartElement("tr"); - writeSimpleElement(xmlStreamWriter, "td", dynamicProperty.name()); - writeSimpleElement(xmlStreamWriter, "td", dynamicProperty.value()); + writeSimpleElement(xmlStreamWriter, "td", dynamicProperty.name(), false, "name"); + writeSimpleElement(xmlStreamWriter, "td", dynamicProperty.value(), false, "value"); xmlStreamWriter.writeStartElement("td"); xmlStreamWriter.writeCharacters(dynamicProperty.description()); if (dynamicProperty.supportsExpressionLanguage()) { From 2c418ece71ee20a2585364458ca4f67d96e7bd5f Mon Sep 17 00:00:00 2001 From: danbress Date: Mon, 23 Mar 2015 09:39:24 -0400 Subject: [PATCH 075/116] NIFI-434 formatting tds to not wrap on spaces --- .../nifi-web-docs/src/main/webapp/css/component-usage.css | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 6aea18b9e5..19f6dffac0 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 @@ -113,7 +113,7 @@ table tr:last-child td:last-child { border-bottom-right-radius:3px; } -td#allowable-values, td#default-value, td#name { +td#allowable-values, td#default-value, td#name, td#value { white-space:nowrap; } From 2429e0540fcce90bfd2ced450d4cd124cff3a6b4 Mon Sep 17 00:00:00 2001 From: Mark Payne Date: Mon, 23 Mar 2015 15:02:03 -0400 Subject: [PATCH 076/116] NIFI-430: Added isExpressionLanguageSupported and isExpressionLanguagePresent methods to ValidationContext --- .../AbstractConfigurableComponent.java | 5 +- .../nifi/components/ValidationContext.java | 19 ++++ .../processor/util/StandardValidators.java | 86 ++++++++++++++++++- .../util/TestStandardValidators.java | 34 ++++---- .../nifi/util/MockValidationContext.java | 27 ++++++ .../processor/StandardValidationContext.java | 25 +++++- 6 files changed, 175 insertions(+), 21 deletions(-) diff --git a/nifi/nifi-api/src/main/java/org/apache/nifi/components/AbstractConfigurableComponent.java b/nifi/nifi-api/src/main/java/org/apache/nifi/components/AbstractConfigurableComponent.java index f4bea5e264..6736788a84 100644 --- a/nifi/nifi-api/src/main/java/org/apache/nifi/components/AbstractConfigurableComponent.java +++ b/nifi/nifi-api/src/main/java/org/apache/nifi/components/AbstractConfigurableComponent.java @@ -187,9 +187,8 @@ public abstract class AbstractConfigurableComponent implements ConfigurableCompo } public final List getPropertyDescriptors() { - final List descriptors = new ArrayList<>(); - descriptors.addAll(getSupportedPropertyDescriptors()); - return descriptors; + final List supported = getSupportedPropertyDescriptors(); + return supported == null ? Collections.emptyList() :new ArrayList<>(supported); } @Override diff --git a/nifi/nifi-api/src/main/java/org/apache/nifi/components/ValidationContext.java b/nifi/nifi-api/src/main/java/org/apache/nifi/components/ValidationContext.java index e50f002f92..b7b72c5141 100644 --- a/nifi/nifi-api/src/main/java/org/apache/nifi/components/ValidationContext.java +++ b/nifi/nifi-api/src/main/java/org/apache/nifi/components/ValidationContext.java @@ -79,4 +79,23 @@ public interface ValidationContext { * @return */ String getAnnotationData(); + + /** + * Returns true if the given value contains a NiFi Expression Language expression, + * false if it does not + * + * @param value + * @return + */ + boolean isExpressionLanguagePresent(String value); + + /** + * Returns true if the property with the given name supports the NiFi Expression Language, + * false if the property does not support the Expression Language or is not a valid property + * name + * + * @param propertyName + * @return + */ + boolean isExpressionLanguageSupported(String propertyName); } diff --git a/nifi/nifi-commons/nifi-processor-utilities/src/main/java/org/apache/nifi/processor/util/StandardValidators.java b/nifi/nifi-commons/nifi-processor-utilities/src/main/java/org/apache/nifi/processor/util/StandardValidators.java index c9ae60974a..62fecf2e90 100644 --- a/nifi/nifi-commons/nifi-processor-utilities/src/main/java/org/apache/nifi/processor/util/StandardValidators.java +++ b/nifi/nifi-commons/nifi-processor-utilities/src/main/java/org/apache/nifi/processor/util/StandardValidators.java @@ -46,7 +46,10 @@ public class StandardValidators { public ValidationResult validate(final String subject, final String input, final ValidationContext context) { final ValidationResult.Builder builder = new ValidationResult.Builder(); builder.subject(subject).input(input); - + if ( context.isExpressionLanguageSupported(subject) && context.isExpressionLanguagePresent(input) ) { + return builder.valid(true).explanation("Contains Expression Language").build(); + } + try { FlowFile.KeyValidator.validateKey(input); builder.valid(true); @@ -63,7 +66,10 @@ public class StandardValidators { public ValidationResult validate(final String subject, final String input, final ValidationContext context) { final ValidationResult.Builder builder = new ValidationResult.Builder(); builder.subject("Property Name").input(subject); - + if ( context.isExpressionLanguageSupported(subject) && context.isExpressionLanguagePresent(input) ) { + return builder.valid(true).explanation("Contains Expression Language").build(); + } + try { FlowFile.KeyValidator.validateKey(subject); builder.valid(true); @@ -78,6 +84,10 @@ public class StandardValidators { public static final Validator POSITIVE_INTEGER_VALIDATOR = new Validator() { @Override public ValidationResult validate(final String subject, final String value, final ValidationContext context) { + if ( context.isExpressionLanguageSupported(subject) && context.isExpressionLanguagePresent(value) ) { + return new ValidationResult.Builder().subject(subject).input(value).explanation("Expression Language Present").valid(true).build(); + } + String reason = null; try { final int intVal = Integer.parseInt(value); @@ -96,6 +106,10 @@ public class StandardValidators { public static final Validator POSITIVE_LONG_VALIDATOR = new Validator() { @Override public ValidationResult validate(final String subject, final String value, final ValidationContext context) { + if ( context.isExpressionLanguageSupported(subject) && context.isExpressionLanguagePresent(value) ) { + return new ValidationResult.Builder().subject(subject).input(value).explanation("Expression Language Present").valid(true).build(); + } + String reason = null; try { final long longVal = Long.parseLong(value); @@ -123,6 +137,10 @@ public class StandardValidators { public static final Validator BOOLEAN_VALIDATOR = new Validator() { @Override public ValidationResult validate(final String subject, final String value, final ValidationContext context) { + if ( context.isExpressionLanguageSupported(subject) && context.isExpressionLanguagePresent(value) ) { + return new ValidationResult.Builder().subject(subject).input(value).explanation("Expression Language Present").valid(true).build(); + } + final boolean valid = "true".equalsIgnoreCase(value) || "false".equalsIgnoreCase(value); final String explanation = valid ? null : "Value must be 'true' or 'false'"; return new ValidationResult.Builder().subject(subject).input(value).valid(valid).explanation(explanation).build(); @@ -132,6 +150,10 @@ public class StandardValidators { public static final Validator INTEGER_VALIDATOR = new Validator() { @Override public ValidationResult validate(final String subject, final String value, final ValidationContext context) { + if ( context.isExpressionLanguageSupported(subject) && context.isExpressionLanguagePresent(value) ) { + return new ValidationResult.Builder().subject(subject).input(value).explanation("Expression Language Present").valid(true).build(); + } + String reason = null; try { Integer.parseInt(value); @@ -146,6 +168,10 @@ public class StandardValidators { public static final Validator LONG_VALIDATOR = new Validator() { @Override public ValidationResult validate(final String subject, final String value, final ValidationContext context) { + if ( context.isExpressionLanguageSupported(subject) && context.isExpressionLanguagePresent(value) ) { + return new ValidationResult.Builder().subject(subject).input(value).explanation("Expression Language Present").valid(true).build(); + } + String reason = null; try { Long.parseLong(value); @@ -160,6 +186,10 @@ public class StandardValidators { public static final Validator NON_NEGATIVE_INTEGER_VALIDATOR = new Validator() { @Override public ValidationResult validate(final String subject, final String value, final ValidationContext context) { + if ( context.isExpressionLanguageSupported(subject) && context.isExpressionLanguagePresent(value) ) { + return new ValidationResult.Builder().subject(subject).input(value).explanation("Expression Language Present").valid(true).build(); + } + String reason = null; try { final int intVal = Integer.parseInt(value); @@ -178,6 +208,10 @@ public class StandardValidators { public static final Validator CHARACTER_SET_VALIDATOR = new Validator() { @Override public ValidationResult validate(final String subject, final String value, final ValidationContext context) { + if ( context.isExpressionLanguageSupported(subject) && context.isExpressionLanguagePresent(value) ) { + return new ValidationResult.Builder().subject(subject).input(value).explanation("Expression Language Present").valid(true).build(); + } + String reason = null; try { if (!Charset.isSupported(value)) { @@ -201,6 +235,10 @@ public class StandardValidators { public static final Validator URI_VALIDATOR = new Validator() { @Override public ValidationResult validate(final String subject, final String input, final ValidationContext context) { + if ( context.isExpressionLanguageSupported(subject) && context.isExpressionLanguagePresent(input) ) { + return new ValidationResult.Builder().subject(subject).input(input).explanation("Expression Language Present").build(); + } + try { new URI(input); return new ValidationResult.Builder().subject(subject).input(input).explanation("Valid URI").valid(true).build(); @@ -215,6 +253,10 @@ public class StandardValidators { public static final Validator ATTRIBUTE_EXPRESSION_LANGUAGE_VALIDATOR = new Validator() { @Override public ValidationResult validate(final String subject, final String input, final ValidationContext context) { + if ( context.isExpressionLanguageSupported(subject) && context.isExpressionLanguagePresent(input) ) { + return new ValidationResult.Builder().subject(subject).input(input).explanation("Expression Language Present").build(); + } + try { context.newExpressionLanguageCompiler().compile(input); return new ValidationResult.Builder().subject(subject).input(input).valid(true).build(); @@ -228,6 +270,10 @@ public class StandardValidators { public static final Validator TIME_PERIOD_VALIDATOR = new Validator() { @Override public ValidationResult validate(final String subject, final String input, final ValidationContext context) { + if ( context.isExpressionLanguageSupported(subject) && context.isExpressionLanguagePresent(input) ) { + return new ValidationResult.Builder().subject(subject).input(input).explanation("Expression Language Present").build(); + } + if (input == null) { return new ValidationResult.Builder().subject(subject).input(input).valid(false).explanation("Time Period cannot be null").build(); } @@ -242,6 +288,10 @@ public class StandardValidators { public static final Validator DATA_SIZE_VALIDATOR = new Validator() { @Override public ValidationResult validate(final String subject, final String input, final ValidationContext context) { + if ( context.isExpressionLanguageSupported(subject) && context.isExpressionLanguagePresent(input) ) { + return new ValidationResult.Builder().subject(subject).input(input).explanation("Expression Language Present").build(); + } + if (input == null) { return new ValidationResult.Builder().subject(subject).input(input).valid(false).explanation("Data Size cannot be null").build(); } @@ -268,6 +318,10 @@ public class StandardValidators { return new Validator() { @Override public ValidationResult validate(final String subject, final String input, final ValidationContext context) { + if ( context.isExpressionLanguageSupported(subject) && context.isExpressionLanguagePresent(input) ) { + return new ValidationResult.Builder().subject(subject).input(input).explanation("Expression Language Present").build(); + } + try { final String evaluatedInput = context.newPropertyValue(input).evaluateAttributeExpressions().getValue(); new URL(evaluatedInput); @@ -292,6 +346,10 @@ public class StandardValidators { @Override public ValidationResult validate(final String subject, final String input, final ValidationContext context) { + if ( context.isExpressionLanguageSupported(subject) && context.isExpressionLanguagePresent(input) ) { + return new ValidationResult.Builder().subject(subject).input(input).explanation("Expression Language Present").build(); + } + final ValidationResult vr = DATA_SIZE_VALIDATOR.validate(subject, input, context); if(!vr.isValid()){ return vr; @@ -313,6 +371,10 @@ public class StandardValidators { return new Validator() { @Override public ValidationResult validate(final String subject, final String input, final ValidationContext context) { + if ( context.isExpressionLanguageSupported(subject) && context.isExpressionLanguagePresent(input) ) { + return new ValidationResult.Builder().subject(subject).input(input).explanation("Expression Language Present").build(); + } + final boolean matches = pattern.matcher(input).matches(); return new ValidationResult.Builder() .input(input) @@ -394,6 +456,10 @@ public class StandardValidators { return new Validator() { @Override public ValidationResult validate(final String subject, final String input, final ValidationContext context) { + if ( context.isExpressionLanguageSupported(subject) && context.isExpressionLanguagePresent(input) ) { + return new ValidationResult.Builder().subject(subject).input(input).explanation("Expression Language Present").build(); + } + String reason = null; try { final long longVal = Long.parseLong(input); @@ -436,6 +502,10 @@ public class StandardValidators { @Override public ValidationResult validate(final String subject, final String input, final ValidationContext context) { + if ( context.isExpressionLanguageSupported(subject) && context.isExpressionLanguagePresent(input) ) { + return new ValidationResult.Builder().subject(subject).input(input).explanation("Expression Language Present").build(); + } + if (input == null) { return new ValidationResult.Builder().subject(subject).input(input).valid(false).explanation("Time Period cannot be null").build(); } @@ -469,6 +539,10 @@ public class StandardValidators { @Override public ValidationResult validate(final String subject, final String value, final ValidationContext context) { + if ( context.isExpressionLanguageSupported(subject) && context.isExpressionLanguagePresent(value) ) { + return new ValidationResult.Builder().subject(subject).input(value).explanation("Expression Language Present").valid(true).build(); + } + final String substituted; if (allowEL) { try { @@ -500,6 +574,10 @@ public class StandardValidators { @Override public ValidationResult validate(final String subject, final String value, final ValidationContext context) { + if ( context.isExpressionLanguageSupported(subject) && context.isExpressionLanguagePresent(value) ) { + return new ValidationResult.Builder().subject(subject).input(value).explanation("Expression Language Present").valid(true).build(); + } + final String substituted; if (allowEL) { try { @@ -541,6 +619,10 @@ public class StandardValidators { return new Validator() { @Override public ValidationResult validate(final String subject, final String input, final ValidationContext context) { + if ( context.isExpressionLanguageSupported(subject) && context.isExpressionLanguagePresent(input) ) { + return new ValidationResult.Builder().subject(subject).input(input).explanation("Expression Language Present").build(); + } + final ControllerService svc = context.getControllerServiceLookup().getControllerService(input); if (svc == null) { diff --git a/nifi/nifi-commons/nifi-processor-utilities/src/test/java/org/apache/nifi/processor/util/TestStandardValidators.java b/nifi/nifi-commons/nifi-processor-utilities/src/test/java/org/apache/nifi/processor/util/TestStandardValidators.java index 70b8d215ac..a8f4bae925 100644 --- a/nifi/nifi-commons/nifi-processor-utilities/src/test/java/org/apache/nifi/processor/util/TestStandardValidators.java +++ b/nifi/nifi-commons/nifi-processor-utilities/src/test/java/org/apache/nifi/processor/util/TestStandardValidators.java @@ -21,10 +21,11 @@ import static org.junit.Assert.assertTrue; import java.util.concurrent.TimeUnit; +import org.apache.nifi.components.ValidationContext; import org.apache.nifi.components.ValidationResult; import org.apache.nifi.components.Validator; - import org.junit.Test; +import org.mockito.Mockito; public class TestStandardValidators { @@ -33,22 +34,24 @@ public class TestStandardValidators { Validator val = StandardValidators.createTimePeriodValidator(1L, TimeUnit.SECONDS, Long.MAX_VALUE, TimeUnit.NANOSECONDS); ValidationResult vr; - vr = val.validate("TimePeriodTest", "0 sense made", null); + final ValidationContext validationContext = Mockito.mock(ValidationContext.class); + + vr = val.validate("TimePeriodTest", "0 sense made", validationContext); assertFalse(vr.isValid()); - vr = val.validate("TimePeriodTest", null, null); + vr = val.validate("TimePeriodTest", null, validationContext); assertFalse(vr.isValid()); - vr = val.validate("TimePeriodTest", "0 secs", null); + vr = val.validate("TimePeriodTest", "0 secs", validationContext); assertFalse(vr.isValid()); - vr = val.validate("TimePeriodTest", "999 millis", null); + vr = val.validate("TimePeriodTest", "999 millis", validationContext); assertFalse(vr.isValid()); - vr = val.validate("TimePeriodTest", "999999999 nanos", null); + vr = val.validate("TimePeriodTest", "999999999 nanos", validationContext); assertFalse(vr.isValid()); - vr = val.validate("TimePeriodTest", "1 sec", null); + vr = val.validate("TimePeriodTest", "1 sec", validationContext); assertTrue(vr.isValid()); } @@ -57,28 +60,29 @@ public class TestStandardValidators { Validator val = StandardValidators.createDataSizeBoundsValidator(100, 1000); ValidationResult vr; - vr = val.validate("DataSizeBounds", "5 GB", null); + final ValidationContext validationContext = Mockito.mock(ValidationContext.class); + vr = val.validate("DataSizeBounds", "5 GB", validationContext); assertFalse(vr.isValid()); - vr = val.validate("DataSizeBounds", "0 B", null); + vr = val.validate("DataSizeBounds", "0 B", validationContext); assertFalse(vr.isValid()); - vr = val.validate("DataSizeBounds", "99 B", null); + vr = val.validate("DataSizeBounds", "99 B", validationContext); assertFalse(vr.isValid()); - vr = val.validate("DataSizeBounds", "100 B", null); + vr = val.validate("DataSizeBounds", "100 B", validationContext); assertTrue(vr.isValid()); - vr = val.validate("DataSizeBounds", "999 B", null); + vr = val.validate("DataSizeBounds", "999 B", validationContext); assertTrue(vr.isValid()); - vr = val.validate("DataSizeBounds", "1000 B", null); + vr = val.validate("DataSizeBounds", "1000 B", validationContext); assertTrue(vr.isValid()); - vr = val.validate("DataSizeBounds", "1001 B", null); + vr = val.validate("DataSizeBounds", "1001 B", validationContext); assertFalse(vr.isValid()); - vr = val.validate("DataSizeBounds", "water", null); + vr = val.validate("DataSizeBounds", "water", validationContext); assertFalse(vr.isValid()); } diff --git a/nifi/nifi-mock/src/main/java/org/apache/nifi/util/MockValidationContext.java b/nifi/nifi-mock/src/main/java/org/apache/nifi/util/MockValidationContext.java index fd0b6cdf4b..c00386ef5d 100644 --- a/nifi/nifi-mock/src/main/java/org/apache/nifi/util/MockValidationContext.java +++ b/nifi/nifi-mock/src/main/java/org/apache/nifi/util/MockValidationContext.java @@ -16,9 +16,13 @@ */ package org.apache.nifi.util; +import java.util.HashMap; +import java.util.List; import java.util.Map; import java.util.Set; +import org.apache.nifi.attribute.expression.language.Query; +import org.apache.nifi.attribute.expression.language.Query.Range; import org.apache.nifi.attribute.expression.language.StandardExpressionLanguageCompiler; import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.components.PropertyValue; @@ -30,9 +34,16 @@ import org.apache.nifi.expression.ExpressionLanguageCompiler; public class MockValidationContext implements ValidationContext, ControllerServiceLookup { private final MockProcessContext context; + private final Map expressionLanguageSupported; public MockValidationContext(final MockProcessContext processContext) { this.context = processContext; + + final Map properties = processContext.getProperties(); + expressionLanguageSupported = new HashMap<>(properties.size()); + for ( final PropertyDescriptor descriptor : properties.keySet() ) { + expressionLanguageSupported.put(descriptor.getName(), descriptor.isExpressionLanguageSupported()); + } } @Override @@ -90,4 +101,20 @@ public class MockValidationContext implements ValidationContext, ControllerServi public boolean isControllerServiceEnabled(final ControllerService service) { return context.isControllerServiceEnabled(service); } + + @Override + public boolean isExpressionLanguagePresent(final String value) { + if ( value == null ) { + return false; + } + + final List elRanges = Query.extractExpressionRanges(value); + return (elRanges != null && !elRanges.isEmpty()); + } + + @Override + public boolean isExpressionLanguageSupported(final String propertyName) { + final Boolean supported = expressionLanguageSupported.get(propertyName); + return Boolean.TRUE.equals(supported); + } } diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/processor/StandardValidationContext.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/processor/StandardValidationContext.java index b216572310..99322be30c 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/processor/StandardValidationContext.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/processor/StandardValidationContext.java @@ -18,10 +18,12 @@ package org.apache.nifi.processor; import java.util.Collections; import java.util.HashMap; +import java.util.List; import java.util.Map; import org.apache.nifi.attribute.expression.language.PreparedQuery; import org.apache.nifi.attribute.expression.language.Query; +import org.apache.nifi.attribute.expression.language.Query.Range; import org.apache.nifi.attribute.expression.language.StandardExpressionLanguageCompiler; import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.components.PropertyValue; @@ -37,6 +39,7 @@ public class StandardValidationContext implements ValidationContext { private final ControllerServiceProvider controllerServiceProvider; private final Map properties; private final Map preparedQueries; + private final Map expressionLanguageSupported; private final String annotationData; public StandardValidationContext(final ControllerServiceProvider controllerServiceProvider, final Map properties, final String annotationData) { @@ -44,7 +47,7 @@ public class StandardValidationContext implements ValidationContext { this.properties = new HashMap<>(properties); this.annotationData = annotationData; - preparedQueries = new HashMap<>(); + preparedQueries = new HashMap<>(properties.size()); for (final Map.Entry entry : properties.entrySet()) { final PropertyDescriptor desc = entry.getKey(); String value = entry.getValue(); @@ -56,6 +59,10 @@ public class StandardValidationContext implements ValidationContext { preparedQueries.put(desc, pq); } + expressionLanguageSupported = new HashMap<>(properties.size()); + for ( final PropertyDescriptor descriptor : properties.keySet() ) { + expressionLanguageSupported.put(descriptor.getName(), descriptor.isExpressionLanguageSupported()); + } } @Override @@ -94,4 +101,20 @@ public class StandardValidationContext implements ValidationContext { public ControllerServiceLookup getControllerServiceLookup() { return controllerServiceProvider; } + + @Override + public boolean isExpressionLanguagePresent(final String value) { + if ( value == null ) { + return false; + } + + final List elRanges = Query.extractExpressionRanges(value); + return (elRanges != null && !elRanges.isEmpty()); + } + + @Override + public boolean isExpressionLanguageSupported(final String propertyName) { + final Boolean supported = expressionLanguageSupported.get(propertyName); + return Boolean.TRUE.equals(supported); + } } From 12fa9e79f767cabc01f6779bc5a3295fa3c595f3 Mon Sep 17 00:00:00 2001 From: Mark Payne Date: Mon, 23 Mar 2015 15:02:42 -0400 Subject: [PATCH 077/116] NIFI-431: Updated validators to use new isExpressionLanguagePresent and isExpressionLanguageSupported methods of ValidationContext --- .../kite/AbstractKiteProcessor.java | 43 ++++++++++--------- .../processors/kite/ConvertCSVToAvro.java | 4 +- .../processors/kite/ConvertJSONToAvro.java | 3 +- .../processors/kite/StoreInKiteDataset.java | 3 +- .../nifi/processors/standard/ControlRate.java | 2 +- .../processors/standard/EvaluateJsonPath.java | 28 +++++++----- .../standard/HandleHttpResponse.java | 2 +- .../nifi/processors/standard/InvokeHTTP.java | 2 +- .../nifi/processors/standard/PostHTTP.java | 1 + 9 files changed, 47 insertions(+), 41 deletions(-) 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 2113854ab0..56418f4bae 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 @@ -78,20 +78,24 @@ abstract class AbstractKiteProcessor extends AbstractProcessor { protected static final Validator RECOGNIZED_URI = new Validator() { @Override - public ValidationResult validate(String subject, String uri, - ValidationContext context) { + public ValidationResult validate(String subject, String uri, ValidationContext context) { String message = "not set"; boolean isValid = true; - if (uri == null || uri.isEmpty()) { + + if (uri.trim().isEmpty()) { isValid = false; - } else if (!uri.contains("$")) { - try { - new URIBuilder(URI.create(uri)).build(); - } catch (RuntimeException e) { - message = e.getMessage(); - isValid = false; + } else { + final boolean elPresent = context.isExpressionLanguageSupported(subject) && context.isExpressionLanguagePresent(uri); + if (!elPresent) { + try { + new URIBuilder(URI.create(uri)).build(); + } catch (RuntimeException e) { + message = e.getMessage(); + isValid = false; + } } } + return new ValidationResult.Builder() .subject(subject) .input(uri) @@ -157,16 +161,16 @@ abstract class AbstractKiteProcessor extends AbstractProcessor { protected static final Validator SCHEMA_VALIDATOR = new Validator() { @Override public ValidationResult validate(String subject, String uri, ValidationContext context) { - Configuration conf = getConfiguration( - context.getProperty(CONF_XML_FILES).getValue()); - + Configuration conf = getConfiguration(context.getProperty(CONF_XML_FILES).getValue()); String error = null; - if (!uri.contains("$")) { - try { - getSchema(uri, conf); - } catch (SchemaNotFoundException e) { - error = e.getMessage(); - } + + final boolean elPresent = context.isExpressionLanguageSupported(subject) && context.isExpressionLanguagePresent(uri); + if (!elPresent) { + try { + getSchema(uri, conf); + } catch (SchemaNotFoundException e) { + error = e.getMessage(); + } } return new ValidationResult.Builder() .subject(subject) @@ -177,8 +181,7 @@ abstract class AbstractKiteProcessor extends AbstractProcessor { } }; - protected static final List ABSTRACT_KITE_PROPS - = ImmutableList.builder() + protected static final List ABSTRACT_KITE_PROPS = ImmutableList.builder() .add(CONF_XML_FILES) .build(); 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 be8c416fcd..c6f58c7bed 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 @@ -88,8 +88,7 @@ public class ConvertCSVToAvro extends AbstractKiteProcessor { static final PropertyDescriptor SCHEMA = new PropertyDescriptor.Builder() .name("Record schema") - .description( - "Outgoing Avro schema for each record created from a CSV row") + .description("Outgoing Avro schema for each record created from a CSV row") .addValidator(SCHEMA_VALIDATOR) .expressionLanguageSupported(true) .required(true) @@ -253,7 +252,6 @@ public class ConvertCSVToAvro extends AbstractKiteProcessor { } catch (ProcessException | DatasetIOException e) { getLogger().error("Failed reading or writing", e); session.transfer(flowFile, FAILURE); - } catch (DatasetException e) { getLogger().error("Failed to read FlowFile", e); session.transfer(flowFile, FAILURE); 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 7e40f14362..7a35e31393 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 @@ -65,8 +65,7 @@ public class ConvertJSONToAvro extends AbstractKiteProcessor { static final PropertyDescriptor SCHEMA = new PropertyDescriptor.Builder() .name("Record schema") - .description( - "Outgoing Avro schema for each record created from a JSON object") + .description("Outgoing Avro schema for each record created from a JSON object") .addValidator(SCHEMA_VALIDATOR) .expressionLanguageSupported(true) .required(true) 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 f65e0bce7c..5586de1c5f 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 @@ -68,8 +68,7 @@ public class StoreInKiteDataset extends AbstractKiteProcessor { public static final PropertyDescriptor KITE_DATASET_URI = new PropertyDescriptor.Builder() .name("Target dataset URI") - .description( - "URI that identifies a Kite dataset where data will be stored") + .description("URI that identifies a Kite dataset where data will be stored") .addValidator(RECOGNIZED_URI) .expressionLanguageSupported(true) .required(true) diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ControlRate.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ControlRate.java index 2c1a8646ea..83febb52de 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ControlRate.java +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ControlRate.java @@ -155,7 +155,7 @@ public class ControlRate extends AbstractProcessor { break; } - final ValidationResult rateResult = rateValidator.validate("Maximum Rate", context.getProperty(MAX_RATE).getValue(), null); + final ValidationResult rateResult = rateValidator.validate("Maximum Rate", context.getProperty(MAX_RATE).getValue(), context); if (!rateResult.isValid()) { validationResults.add(rateResult); } diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateJsonPath.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateJsonPath.java index e32eb126a0..2634da2913 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateJsonPath.java +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/EvaluateJsonPath.java @@ -16,10 +16,19 @@ */ package org.apache.nifi.processors.standard; -import com.jayway.jsonpath.DocumentContext; -import com.jayway.jsonpath.InvalidJsonException; -import com.jayway.jsonpath.JsonPath; -import com.jayway.jsonpath.PathNotFoundException; +import java.io.IOException; +import java.io.OutputStream; +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; import org.apache.commons.lang3.StringUtils; import org.apache.nifi.annotation.behavior.DynamicProperty; @@ -27,7 +36,6 @@ import org.apache.nifi.annotation.behavior.EventDriven; import org.apache.nifi.annotation.behavior.SideEffectFree; import org.apache.nifi.annotation.behavior.SupportsBatching; import org.apache.nifi.annotation.documentation.CapabilityDescription; -import org.apache.nifi.annotation.behavior.DynamicRelationship; import org.apache.nifi.annotation.documentation.Tags; import org.apache.nifi.annotation.lifecycle.OnRemoved; import org.apache.nifi.components.PropertyDescriptor; @@ -44,12 +52,10 @@ import org.apache.nifi.processor.io.OutputStreamCallback; import org.apache.nifi.stream.io.BufferedOutputStream; import org.apache.nifi.util.ObjectHolder; -import java.io.IOException; -import java.io.OutputStream; -import java.nio.charset.StandardCharsets; -import java.util.*; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ConcurrentMap; +import com.jayway.jsonpath.DocumentContext; +import com.jayway.jsonpath.InvalidJsonException; +import com.jayway.jsonpath.JsonPath; +import com.jayway.jsonpath.PathNotFoundException; @EventDriven @SideEffectFree diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/HandleHttpResponse.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/HandleHttpResponse.java index b50ce3bae6..bfe53efa90 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/HandleHttpResponse.java +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/HandleHttpResponse.java @@ -54,7 +54,7 @@ public class HandleHttpResponse extends AbstractProcessor { .name("HTTP Status Code") .description("The HTTP Status Code to use when responding to the HTTP Request. See Section 10 of RFC 2616 for more information.") .required(true) - .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) + .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR) .expressionLanguageSupported(true) .build(); public static final PropertyDescriptor HTTP_CONTEXT_MAP = new PropertyDescriptor.Builder() diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/InvokeHTTP.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/InvokeHTTP.java index 63ec0bbdbe..08384d988a 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/InvokeHTTP.java +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/InvokeHTTP.java @@ -203,7 +203,7 @@ public final class InvokeHTTP extends AbstractProcessor { .description("Remote URL which will be connected to, including scheme, host, port, path.") .required(true) .expressionLanguageSupported(true) - .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) + .addValidator(StandardValidators.URL_VALIDATOR) .build(); PropertyDescriptor PROP_CONNECT_TIMEOUT = new PropertyDescriptor.Builder() diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PostHTTP.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PostHTTP.java index 5f10772df9..d67ed092d9 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PostHTTP.java +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PostHTTP.java @@ -146,6 +146,7 @@ public class PostHTTP extends AbstractProcessor { .description("The URL to POST to. The first part of the URL must be static. However, the path of the URL may be defined using the Attribute Expression Language. For example, https://${hostname} is not valid, but https://1.1.1.1:8080/files/${nf.file.name} is valid.") .required(true) .addValidator(StandardValidators.createRegexMatchingValidator(Pattern.compile("https?\\://.*"))) + .addValidator(StandardValidators.URL_VALIDATOR) .expressionLanguageSupported(true) .build(); public static final PropertyDescriptor SEND_AS_FLOWFILE = new PropertyDescriptor.Builder() From d6408046bca336e4a11a9c123eef38139f05af78 Mon Sep 17 00:00:00 2001 From: Mark Payne Date: Mon, 23 Mar 2015 15:11:41 -0400 Subject: [PATCH 078/116] NIFI-408: Instead of yielding for 100 milliseconds when data is not available, Root Group Port should poll with a max wait time of 100 milliseconds and if no data is available return without yielding. --- .../org/apache/nifi/remote/StandardRootGroupPort.java | 11 ++++++++--- 1 file changed, 8 insertions(+), 3 deletions(-) diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/StandardRootGroupPort.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/StandardRootGroupPort.java index 5fbd92fd6b..021531fec2 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/StandardRootGroupPort.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/StandardRootGroupPort.java @@ -28,6 +28,7 @@ import java.util.Map; import java.util.Set; import java.util.concurrent.ArrayBlockingQueue; import java.util.concurrent.BlockingQueue; +import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicReference; import java.util.concurrent.locks.Lock; @@ -63,7 +64,6 @@ import org.apache.nifi.reporting.BulletinRepository; import org.apache.nifi.reporting.Severity; import org.apache.nifi.scheduling.SchedulingStrategy; import org.apache.nifi.user.NiFiUser; - import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -126,9 +126,14 @@ public class StandardRootGroupPort extends AbstractPort implements RootGroupPort @Override public void onTrigger(final ProcessContext context, final ProcessSessionFactory sessionFactory) { - final FlowFileRequest flowFileRequest = requestQueue.poll(); + final FlowFileRequest flowFileRequest; + try { + flowFileRequest = requestQueue.poll(100, TimeUnit.MILLISECONDS); + } catch (final InterruptedException ie) { + return; + } + if ( flowFileRequest == null ) { - context.yield(); return; } From 54f3476a4cc975f52d543a53f1f9aec97ba3b858 Mon Sep 17 00:00:00 2001 From: Mark Payne Date: Mon, 23 Mar 2015 15:19:18 -0400 Subject: [PATCH 079/116] NIFI-450: Catch Throwable from all implementations of Runnable in the FileSystemRepository; these are expected to always be running, so if anything odd like an OutOfMemoryError occurs, this needs to be caught rather than allowing the thread to die --- .../repository/FileSystemRepository.java | 78 ++++++++++--------- 1 file changed, 41 insertions(+), 37 deletions(-) diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/FileSystemRepository.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/FileSystemRepository.java index a3e24c4bd6..2c3751b060 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/FileSystemRepository.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/FileSystemRepository.java @@ -791,34 +791,38 @@ public class FileSystemRepository implements ContentRepository { @Override public void run() { - // Get all of the Destructable Claims and bin them based on their Container. We do this - // because the Container generally maps to a physical partition on the disk, so we want a few - // different threads hitting the different partitions but don't want multiple threads hitting - // the same partition. - final List toDestroy = new ArrayList<>(); - while (true) { - toDestroy.clear(); - contentClaimManager.drainDestructableClaims(toDestroy, 10000); - if (toDestroy.isEmpty()) { - return; - } - - for (final ContentClaim claim : toDestroy) { - final String container = claim.getContainer(); - final BlockingQueue claimQueue = reclaimable.get(container); - - try { - while (true) { - if (claimQueue.offer(claim, 10, TimeUnit.MINUTES)) { - break; - } else { - LOG.warn("Failed to clean up {} because old claims aren't being cleaned up fast enough. This Content Claim will remain in the Content Repository until NiFi is restarted, at which point it will be cleaned up", claim); + try { + // Get all of the Destructable Claims and bin them based on their Container. We do this + // because the Container generally maps to a physical partition on the disk, so we want a few + // different threads hitting the different partitions but don't want multiple threads hitting + // the same partition. + final List toDestroy = new ArrayList<>(); + while (true) { + toDestroy.clear(); + contentClaimManager.drainDestructableClaims(toDestroy, 10000); + if (toDestroy.isEmpty()) { + return; + } + + for (final ContentClaim claim : toDestroy) { + final String container = claim.getContainer(); + final BlockingQueue claimQueue = reclaimable.get(container); + + try { + while (true) { + if (claimQueue.offer(claim, 10, TimeUnit.MINUTES)) { + break; + } else { + LOG.warn("Failed to clean up {} because old claims aren't being cleaned up fast enough. This Content Claim will remain in the Content Repository until NiFi is restarted, at which point it will be cleaned up", claim); + } } + } catch (final InterruptedException ie) { + LOG.warn("Failed to clean up {} because thread was interrupted", claim); } - } catch (final InterruptedException ie) { - LOG.warn("Failed to clean up {} because thread was interrupted", claim); } } + } catch (final Throwable t) { + LOG.error("Failed to cleanup content claims due to {}", t); } } } @@ -1198,23 +1202,23 @@ public class FileSystemRepository implements ContentRepository { @Override public void run() { - if (oldestArchiveDate.get() > (System.currentTimeMillis() - maxArchiveMillis)) { - final Long minRequiredSpace = minUsableContainerBytesForArchive.get(containerName); - if (minRequiredSpace == null) { - return; - } - - try { - final long usableSpace = getContainerUsableSpace(containerName); - if (usableSpace > minRequiredSpace) { + try { + if (oldestArchiveDate.get() > (System.currentTimeMillis() - maxArchiveMillis)) { + final Long minRequiredSpace = minUsableContainerBytesForArchive.get(containerName); + if (minRequiredSpace == null) { return; } - } catch (final Exception e) { - LOG.error("Failed to determine space available in container {}; will attempt to cleanup archive", containerName); + + try { + final long usableSpace = getContainerUsableSpace(containerName); + if (usableSpace > minRequiredSpace) { + return; + } + } catch (final Exception e) { + LOG.error("Failed to determine space available in container {}; will attempt to cleanup archive", containerName); + } } - } - try { Thread.currentThread().setName("Cleanup Archive for " + containerName); final long oldestContainerArchive; From a7b826ae20588705868749390187852f2c05e0ff Mon Sep 17 00:00:00 2001 From: Mark Payne Date: Mon, 23 Mar 2015 15:39:53 -0400 Subject: [PATCH 080/116] NIFI-268: If processor is using deprecated annotations, should fail unit test --- .../org/apache/nifi/util/StandardProcessorTestRunner.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/nifi/nifi-mock/src/main/java/org/apache/nifi/util/StandardProcessorTestRunner.java b/nifi/nifi-mock/src/main/java/org/apache/nifi/util/StandardProcessorTestRunner.java index f3fd3e81e5..8d691dd879 100644 --- a/nifi/nifi-mock/src/main/java/org/apache/nifi/util/StandardProcessorTestRunner.java +++ b/nifi/nifi-mock/src/main/java/org/apache/nifi/util/StandardProcessorTestRunner.java @@ -137,14 +137,14 @@ public class StandardProcessorTestRunner implements TestRunner { private static void detectDeprecatedAnnotations(final Processor processor) { for ( final Class annotationClass : deprecatedTypeAnnotations ) { if ( processor.getClass().isAnnotationPresent(annotationClass) ) { - logger.warn("Processor is using deprecated Annotation " + annotationClass.getCanonicalName()); + Assert.fail("Processor is using deprecated Annotation " + annotationClass.getCanonicalName()); } } for ( final Class annotationClass : deprecatedMethodAnnotations ) { for ( final Method method : processor.getClass().getMethods() ) { if ( method.isAnnotationPresent(annotationClass) ) { - logger.warn("Processor is using deprecated Annotation " + annotationClass.getCanonicalName() + " for method " + method); + Assert.fail("Processor is using deprecated Annotation " + annotationClass.getCanonicalName() + " for method " + method); } } } From 06ee1291c2e4a9fbfc1d79d19e426ffff9ade232 Mon Sep 17 00:00:00 2001 From: danbress Date: Tue, 24 Mar 2015 11:56:02 -0400 Subject: [PATCH 081/116] NIFI-460 using identifiesControllerService --- .../nifi/processors/standard/InvokeHTTP.java | 29 +++++++------------ 1 file changed, 10 insertions(+), 19 deletions(-) diff --git a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/InvokeHTTP.java b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/InvokeHTTP.java index 08384d988a..c8345d2825 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/InvokeHTTP.java +++ b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/InvokeHTTP.java @@ -85,19 +85,10 @@ public final class InvokeHTTP extends AbstractProcessor { //-- properties --// @Override protected List getSupportedPropertyDescriptors() { - Set contextIdentifiers = getControllerServiceLookup().getControllerServiceIdentifiers(SSLContextService.class); - PropertyDescriptor contextServiceSelector = new PropertyDescriptor.Builder() - .fromPropertyDescriptor(Config.PROP_SSL_CONTEXT_SERVICE) - .allowableValues(contextIdentifiers) - .build(); - - List list = new ArrayList<>(Config.PROPERTIES); - list.add(2, contextServiceSelector); - - return Collections.unmodifiableList(list); + return Config.PROPERTIES; } - + @Override protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(String propertyDescriptorName) { if (Config.PROP_TRUSTED_HOSTNAME.getName().equalsIgnoreCase(propertyDescriptorName)) { @@ -248,9 +239,17 @@ public final class InvokeHTTP extends AbstractProcessor { .addValidator(StandardValidators.REGULAR_EXPRESSION_VALIDATOR) .build(); + PropertyDescriptor PROP_SSL_CONTEXT_SERVICE = new PropertyDescriptor.Builder() + .name("SSL Context Service") + .description("The SSL Context Service used to provide client certificate information for TLS/SSL (https) connections.") + .required(false) + .identifiesControllerService(SSLContextService.class) + .build(); + List PROPERTIES = Collections.unmodifiableList(Arrays.asList( PROP_METHOD, PROP_URL, + PROP_SSL_CONTEXT_SERVICE, PROP_CONNECT_TIMEOUT, PROP_READ_TIMEOUT, PROP_DATE_HEADER, @@ -258,14 +257,6 @@ public final class InvokeHTTP extends AbstractProcessor { PROP_ATTRIBUTES_TO_SEND )); - // The allowableValues of the SSL Context Service property is dynamically populated at run time. - PropertyDescriptor PROP_SSL_CONTEXT_SERVICE = new PropertyDescriptor.Builder() - .name("SSL Context Service") - .description("The SSL Context Service used to provide client certificate information for TLS/SSL (https) connections.") - .required(false) - .addValidator(StandardValidators.createControllerServiceExistsValidator(SSLContextService.class)) - .build(); - // property to allow the hostname verifier to be overridden // this is a "hidden" property - it's configured using a dynamic user property PropertyDescriptor PROP_TRUSTED_HOSTNAME = new PropertyDescriptor.Builder() From a5e140f10705e119c6aa7d52be03f01b94b75709 Mon Sep 17 00:00:00 2001 From: danbress Date: Tue, 24 Mar 2015 14:18:56 -0400 Subject: [PATCH 082/116] NIFI-460 deprecating StandardValidators.createControllerServiceExistsValidator --- .../apache/nifi/processor/util/StandardValidators.java | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/nifi/nifi-commons/nifi-processor-utilities/src/main/java/org/apache/nifi/processor/util/StandardValidators.java b/nifi/nifi-commons/nifi-processor-utilities/src/main/java/org/apache/nifi/processor/util/StandardValidators.java index 62fecf2e90..a22e7bb249 100644 --- a/nifi/nifi-commons/nifi-processor-utilities/src/main/java/org/apache/nifi/processor/util/StandardValidators.java +++ b/nifi/nifi-commons/nifi-processor-utilities/src/main/java/org/apache/nifi/processor/util/StandardValidators.java @@ -615,6 +615,14 @@ public class StandardValidators { } } + /** + * Creates a validator based on existence of a {@link ControllerService}. + * + * @param serviceClass the controller service API your {@link ConfigurableComponent} depends on + * @return a Validator + * @deprecated As of release 0.1.0-incubating, replaced by {@link org.apache.nifi.components.PropertyDescriptor.Builder#identifiesControllerService(Class)} + */ + @Deprecated public static Validator createControllerServiceExistsValidator(final Class serviceClass) { return new Validator() { @Override From 3a27c378cc143abc5e59dfb746c5e0d09341b185 Mon Sep 17 00:00:00 2001 From: Tony Kurc Date: Wed, 25 Mar 2015 21:52:23 -0400 Subject: [PATCH 083/116] NIFI-464 This closes #45 (minor revision - replace hard coded constant with length) --- .../nifi/distributed/cache/protocol/ProtocolHandshake.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/nifi/nifi-nar-bundles/nifi-standard-services/nifi-distributed-cache-services-bundle/nifi-distributed-cache-protocol/src/main/java/org/apache/nifi/distributed/cache/protocol/ProtocolHandshake.java b/nifi/nifi-nar-bundles/nifi-standard-services/nifi-distributed-cache-services-bundle/nifi-distributed-cache-protocol/src/main/java/org/apache/nifi/distributed/cache/protocol/ProtocolHandshake.java index da2acade49..55cf51f28d 100644 --- a/nifi/nifi-nar-bundles/nifi-standard-services/nifi-distributed-cache-services-bundle/nifi-distributed-cache-protocol/src/main/java/org/apache/nifi/distributed/cache/protocol/ProtocolHandshake.java +++ b/nifi/nifi-nar-bundles/nifi-standard-services/nifi-distributed-cache-services-bundle/nifi-distributed-cache-protocol/src/main/java/org/apache/nifi/distributed/cache/protocol/ProtocolHandshake.java @@ -53,7 +53,7 @@ public class ProtocolHandshake { final DataOutputStream dos = new DataOutputStream(out); try { - final byte[] magicHeaderBuffer = new byte[4]; + final byte[] magicHeaderBuffer = new byte[MAGIC_HEADER.length]; dis.readFully(magicHeaderBuffer); receiveVersionNegotiation(versionNegotiator, dis, dos); From 42a4f9024262c2b6e1871d6406e81ba72695f9ab Mon Sep 17 00:00:00 2001 From: Mark Payne Date: Fri, 27 Mar 2015 14:46:20 -0400 Subject: [PATCH 084/116] NIFI-443: If a funnel is given an incoming connection but has no outgoing connection, it continually runs and fails, logging a lot of ERROR messages --- .../nifi/controller/tasks/ContinuallyRunConnectableTask.java | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/tasks/ContinuallyRunConnectableTask.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/tasks/ContinuallyRunConnectableTask.java index 408032cbf3..a1219a47ac 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/tasks/ContinuallyRunConnectableTask.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/tasks/ContinuallyRunConnectableTask.java @@ -21,6 +21,7 @@ import java.util.concurrent.atomic.AtomicLong; import org.apache.nifi.annotation.lifecycle.OnStopped; import org.apache.nifi.connectable.Connectable; +import org.apache.nifi.connectable.ConnectableType; import org.apache.nifi.controller.repository.StandardProcessSessionFactory; import org.apache.nifi.controller.scheduling.ProcessContextFactory; import org.apache.nifi.controller.scheduling.ScheduleState; @@ -69,7 +70,9 @@ public class ContinuallyRunConnectableTask implements Callable { final boolean triggerWhenEmpty = connectable.isTriggerWhenEmpty(); boolean flowFilesQueued = true; final boolean shouldRun = (connectable.getYieldExpiration() < System.currentTimeMillis()) - && (triggerWhenEmpty || (flowFilesQueued = Connectables.flowFilesQueued(connectable))) && (connectable.getRelationships().isEmpty() || Connectables.anyRelationshipAvailable(connectable)); + && (triggerWhenEmpty || (flowFilesQueued = Connectables.flowFilesQueued(connectable))) + && (connectable.getConnectableType() != ConnectableType.FUNNEL || !connectable.getConnections().isEmpty()) + && (connectable.getRelationships().isEmpty() || Connectables.anyRelationshipAvailable(connectable)); if (shouldRun) { scheduleState.incrementActiveThreadCount(); From 909ded32aaa4cc5f70b162792e9a4216357826e1 Mon Sep 17 00:00:00 2001 From: Mark Payne Date: Fri, 27 Mar 2015 15:06:11 -0400 Subject: [PATCH 085/116] NIFI-443: Fixed out-of-date comments/documentation --- .../tasks/ContinuallyRunConnectableTask.java | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/tasks/ContinuallyRunConnectableTask.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/tasks/ContinuallyRunConnectableTask.java index a1219a47ac..b5b60c9bf8 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/tasks/ContinuallyRunConnectableTask.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/tasks/ContinuallyRunConnectableTask.java @@ -62,11 +62,11 @@ public class ContinuallyRunConnectableTask implements Callable { } // Connectable should run if the following conditions are met: - // 1. It's an Input Port or or is a Remote Input Port or has incoming FlowFiles queued - // 2. Any relationship is available (since there's only 1 - // relationship for a Connectable, we can say "any" or "all" and - // it means the same thing) - // 3. It is not yielded. + // 1. It is not yielded. + // 2. It has incoming connections with FlowFiles queued or doesn't expect incoming connections + // 3. If it is a funnel, it has an outgoing connection (this is needed because funnels are "always on"; other + // connectable components cannot even be started if they need an outbound connection and don't have one) + // 4. There is a connection for each relationship. final boolean triggerWhenEmpty = connectable.isTriggerWhenEmpty(); boolean flowFilesQueued = true; final boolean shouldRun = (connectable.getYieldExpiration() < System.currentTimeMillis()) From ed5fd733d8a80b5adbaed7389e9369c192a6f07b Mon Sep 17 00:00:00 2001 From: Jenn Barnabee Date: Mon, 23 Mar 2015 05:57:08 -0400 Subject: [PATCH 086/116] NIFI-422 Updated Clustering Configuration section to account for some missing details; added short troubleshooting section. Signed-off-by: joewitt --- .../main/asciidoc/administration-guide.adoc | 21 +++++++++++++++---- 1 file changed, 17 insertions(+), 4 deletions(-) diff --git a/nifi/nifi-docs/src/main/asciidoc/administration-guide.adoc b/nifi/nifi-docs/src/main/asciidoc/administration-guide.adoc index 96dd423619..d33b4de975 100644 --- a/nifi/nifi-docs/src/main/asciidoc/administration-guide.adoc +++ b/nifi/nifi-docs/src/main/asciidoc/administration-guide.adoc @@ -229,7 +229,7 @@ When the DFM makes changes to the dataflow, the NCM communicates those changes t *Dealing with Disconnected Nodes* + -A DFM may manually disconnect a node from the cluster. But if a node becomes disconnected for any other reason (such as due to lack of heartbeat), the NCM will show a bulletin on the User Interface, and the DFM will not be able to make any changes to the dataflow until the issue of the disconnected node is resolved. The DFM or the Administrator will need to troubleshoot the issue with the node and resolve it before any new changes may be made to the dataflow. However, it is worth noting that just because a node is disconnected does not mean that it is definitely down; it just means that the NCM cannot communicate with the node. +A DFM may manually disconnect a node from the cluster. But if a node becomes disconnected for any other reason (such as due to lack of heartbeat), the NCM will show a bulletin on the User Interface, and the DFM will not be able to make any changes to the dataflow until the issue of the disconnected node is resolved. The DFM or the Administrator will need to troubleshoot the issue with the node and resolve it before any new changes may be made to the dataflow. However, it is worth noting that just because a node is disconnected does not mean that it is not working; it just means that the NCM cannot communicate with the node. *Basic Cluster Setup* + @@ -242,7 +242,7 @@ This section describes the setup for a simple two-node, non-secure, unicast clus Administrators may install each instance on a separate server; however, it is also perfectly fine to install the NCM and one of the nodes on the same server, as the NCM is very lightweight. Just keep in mind that the ports assigned to each instance must not collide if the NCM and one of the nodes share the same server. -For each instance, the clustering properties in the _nifi.properties_ file will need to be updated. All the clustering properties are described in the <> section of this guide; however, in this section, we will focus on the minimum properties that must be set for simple cluster. +For each instance, certain properties in the _nifi.properties_ file will need to be updated. In particular, the Web and Clustering properties should be evaluated for your situation and adjusted accordingly. All the properties are described in the <> section of this guide; however, in this section, we will focus on the minimum properties that must be set for a simple cluster. For all three instances, the Cluster Common Properties can be left with the default settings. Note, however, that if you change these settings, they must be set the same on every instance in the cluster (NCM and nodes). @@ -255,18 +255,20 @@ For the NCM, the minimum properties to configure are as follows: For Node 1, the minimum properties to configure are as follows: -* Under the Web Properties, set either the http or https port that you want Node 1 to run on. If the NCM is running on the same server, choose a different web port for Node 1. +* Under the Web Properties, set either the http or https port that you want Node 1 to run on. If the NCM is running on the same server, choose a different web port for Node 1. Also, consider whether you need to set the http or https host property. * Under Cluster Node Properties, set the following: ** nifi.cluster.is.node - Set this to _true_. +** nifi.cluster.node.address - Set this to the fully qualified hostname of the node. If left blank, it defaults to "localhost". ** nifi.cluster.node.protocol.port - Set this to an open port that is higher than 1024 (anything lower requires root). If Node 1 and the NCM are on the same server, make sure this port is different from the nifi.cluster.protocol.manager.port. ** nifi.cluster.node.unicast.manager.address - Set this to the NCM's fully qualified hostname. ** nifi.cluster.node.unicast.manager.protocol.port - Set this to exactly the same port that was set on the NCM for the property nifi.cluster.manager.protocol.port. For Node 2, the minimum properties to configure are as follows: -* Under the Web Properties, set either the http or https port that you want Node 2 to run on. +* Under the Web Properties, set either the http or https port that you want Node 2 to run on. Also, consider whether you need to set the http or https host property. * Under the Cluster Node Properties, set the following: ** nifi.cluster.is.node - Set this to _true_. +** nifi.cluster.node.address - Set this to the fully qualified hostname of the node. If left blank, it defaults to "localhost". ** nifi.cluster.node.protocol.port - Set this to an open port that is higher than 1024 (anything lower requires root). ** nifi.cluster.node.unicast.manager.address - Set this to the NCM's fully qualified hostname. ** nifi.cluster.node.unicast.manager.protocol.port - Set this to exactly the same port that was set on the NCM for the property nifi.cluster.manager.protocol.port. @@ -275,6 +277,17 @@ Now, it is possible to start up the cluster. Technically, it does not matter whi image:ncm.png["NCM User Interface", width=940] +*Troubleshooting* + +If you encounter issues and your cluster does not work as described, investigate the nifi.app log and nifi.user log on both the NCM and the nodes. If needed, you can change the logging level to DEBUG by editing the conf/logback.xml file. Specifically, set the level="DEBUG" in the following line (instead of "INFO"): + +---- + + + +---- + [[system_properties]] From ac409fd69aefedbfa72f90d1647da4309cf46dfc Mon Sep 17 00:00:00 2001 From: Brandon DeVries Date: Mon, 30 Mar 2015 12:11:15 -0400 Subject: [PATCH 087/116] NIFI-424 Processors / Controller Services should not be able to directly call ControllerService.initialize and ControllerService.onPropertyModified --- .../StandardControllerServiceProvider.java | 6 ++ ...StandardControllerServiceProviderTest.java | 71 +++++++++++++++++++ .../service/util/TestControllerService.java | 61 ++++++++++++++++ ...g.apache.nifi.controller.ControllerService | 15 ++++ 4 files changed, 153 insertions(+) create mode 100644 nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/StandardControllerServiceProviderTest.java create mode 100644 nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/util/TestControllerService.java create mode 100644 nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/resources/META-INF/services/org.apache.nifi.controller.ControllerService diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceProvider.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceProvider.java index a40e43e5e6..7a8e22fd4a 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceProvider.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceProvider.java @@ -117,6 +117,12 @@ public class StandardControllerServiceProvider implements ControllerServiceProvi final InvocationHandler invocationHandler = new InvocationHandler() { @Override public Object invoke(final Object proxy, final Method method, final Object[] args) throws Throwable { + + final String methodName = method.getName(); + if("initialize".equals(methodName) || "onPropertyModified".equals(methodName)){ + throw new UnsupportedOperationException(method + " may only be invoked by the NiFi framework"); + } + final ControllerServiceNode node = serviceNodeHolder.get(); if (node.isDisabled() && !validDisabledMethods.contains(method)) { // Use nar class loader here because we are implicitly calling toString() on the original implementation. diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/StandardControllerServiceProviderTest.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/StandardControllerServiceProviderTest.java new file mode 100644 index 0000000000..b889bc81c1 --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/StandardControllerServiceProviderTest.java @@ -0,0 +1,71 @@ +/* + * 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.controller.service; + +import org.apache.nifi.controller.ControllerService; +import org.apache.nifi.controller.StandardFlowServiceTest; +import org.apache.nifi.nar.ExtensionManager; +import org.apache.nifi.nar.NarClassLoaders; +import org.apache.nifi.reporting.InitializationException; +import org.apache.nifi.util.NiFiProperties; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Test; + +public class StandardControllerServiceProviderTest { + + private ControllerService proxied; + private ControllerService implementation; + + @BeforeClass + public static void setupSuite() throws Exception { + System.setProperty(NiFiProperties.PROPERTIES_FILE_PATH, StandardFlowServiceTest.class.getResource("/conf/nifi.properties").getFile()); + NiFiProperties properties = NiFiProperties.getInstance(); + NarClassLoaders.load(properties); + ExtensionManager.discoverExtensions(); + } + + @Before + public void setup() throws Exception { + String id = "id"; + String clazz = "org.apache.nifi.controller.service.util.TestControllerService"; + ControllerServiceProvider provider = new StandardControllerServiceProvider(); + ControllerServiceNode node = provider.createControllerService(clazz,id,true); + proxied = node.getProxiedControllerService(); + implementation = node.getControllerServiceImplementation(); + } + + @Test (expected=UnsupportedOperationException.class) + public void testCallProxiedOnPropertyModified() { + proxied.onPropertyModified(null, "oldValue", "newValue"); + } + + @Test + public void testCallImplementationOnPropertyModified() { + implementation.onPropertyModified(null, "oldValue", "newValue"); + } + + @Test (expected=UnsupportedOperationException.class) + public void testCallProxiedInitialized() throws InitializationException { + proxied.initialize(null); + } + + @Test + public void testCallImplementationInitialized() throws InitializationException { + implementation.initialize(null); + } +} diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/util/TestControllerService.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/util/TestControllerService.java new file mode 100644 index 0000000000..95200a0eeb --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/util/TestControllerService.java @@ -0,0 +1,61 @@ +/* + * 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.controller.service.util; + +import java.util.Collection; +import java.util.List; + +import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.components.ValidationContext; +import org.apache.nifi.components.ValidationResult; +import org.apache.nifi.controller.ControllerService; +import org.apache.nifi.controller.ControllerServiceInitializationContext; +import org.apache.nifi.reporting.InitializationException; + +public class TestControllerService implements ControllerService { + + @Override + public Collection validate(ValidationContext context) { + return null; + } + + @Override + public PropertyDescriptor getPropertyDescriptor(String name) { + return null; + } + + @Override + public void onPropertyModified(PropertyDescriptor descriptor, + String oldValue, String newValue) { + } + + @Override + public List getPropertyDescriptors() { + return null; + } + + @Override + public String getIdentifier() { + return null; + } + + @Override + public void initialize(ControllerServiceInitializationContext context) + throws InitializationException { + } + +} diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/resources/META-INF/services/org.apache.nifi.controller.ControllerService b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/resources/META-INF/services/org.apache.nifi.controller.ControllerService new file mode 100644 index 0000000000..bd6a4f3935 --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/resources/META-INF/services/org.apache.nifi.controller.ControllerService @@ -0,0 +1,15 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +org.apache.nifi.controller.service.util.TestControllerService From e9647717e39d4046092ff669e5ccbcd0c2ffbd89 Mon Sep 17 00:00:00 2001 From: Matt Gilman Date: Tue, 31 Mar 2015 16:45:05 -0400 Subject: [PATCH 088/116] Squashed commit of the following: commit 9cfc3ab42fbe46af113a1f0db41aa95e6e0b4568 Author: Matt Gilman Date: Tue Mar 31 16:32:13 2015 -0400 NIFI-250: - Updating test to fix compilation error after merge. commit 266d44b11c92cea51f882483adcfa898feeaac44 Merge: 97632fb e7d6d94 Author: Matt Gilman Date: Tue Mar 31 16:11:12 2015 -0400 Merge branch 'develop' into NIFI-250 commit 97632fbce67850c66b765f667ec2a1ed6e540914 Author: Matt Gilman Date: Tue Mar 31 12:12:01 2015 -0400 NIFI-250: - Removing extra call to show configuration dialog. commit 23465a1e116ee10c2c73a9ed25d7b382096f9b52 Merge: d4321f5 1abee29 Author: Matt Gilman Date: Tue Mar 31 11:09:32 2015 -0400 Merge branch 'NIFI-250' of https://git-wip-us.apache.org/repos/asf/incubator-nifi into NIFI-250 commit 1abee2964380bf2aee91189f471035fe3df92919 Author: Mark Payne Date: Tue Mar 31 10:58:53 2015 -0400 NIFI-250: Fixed bugs with configuring reporting tasks on restart commit 712327fe8eac0c96a4d676a1ffe2210801d97200 Merge: abd1dc3 1cee436 Author: Mark Payne Date: Tue Mar 31 08:41:56 2015 -0400 Merge branch 'NIFI-250' of http://git-wip-us.apache.org/repos/asf/incubator-nifi into NIFI-250 commit abd1dc3362ef6297eb3f044aadc58c181114fe06 Author: Mark Payne Date: Tue Mar 31 08:41:45 2015 -0400 NIFI-250: Fixed bug that caused IllegalArgumentException when loading reporting tasks on NCM if property is not set commit d4321f50d6adc1534d318405fb23838032c054d7 Author: Matt Gilman Date: Tue Mar 31 07:39:47 2015 -0400 NIFI-250: - Updating Javadocs. commit 5394a826b07bb897299bebefe275f633c02e95ce Author: Matt Gilman Date: Tue Mar 31 07:30:57 2015 -0400 NIFI-250: - Updating messages included in exceptions during error conditions when interacting with a custom UI. commit 1cee4367888d899d9396f7e48f2badfa8e11f07c Author: Matt Gilman Date: Mon Mar 30 14:54:16 2015 -0400 NIFI-250: - Fixing the width of the referencing components when the content overflows. commit b32e2712c1735d57bf1d34debb0268e42a544c56 Author: Matt Gilman Date: Mon Mar 30 14:19:49 2015 -0400 NIFI-250: - Using a List when (de)serializing allowable values. This is preferable than calling out a specific Set implementation. commit 2949beb6f57a7af469257a27adffa9397e234d7b Author: Matt Gilman Date: Mon Mar 30 13:20:55 2015 -0400 NIFI-250: - Only recording previously configured controller service when attempting to change a controller service. - Only polling schedulable components when stopping. No need to poll during start requests. commit 0b77160da76877b35a5777b0bcf552d5985ba64d Author: Matt Gilman Date: Mon Mar 30 13:17:44 2015 -0400 NIFI-250: - Adding tooltip to referencing components section of the enable/disable controller service dialog. - Fixing typo. commit 37d2bd1e1e564c516cb2d96c7f1b361c9034110a Author: Matt Gilman Date: Mon Mar 30 12:10:44 2015 -0400 NIFI-250: - Fixing controller service ancestor calculation. commit 2673c39a62462e5ade8363b379424241029f2058 Author: Matt Gilman Date: Mon Mar 30 12:09:37 2015 -0400 NIFI-250: - Correcting the number of displayed controller service types. - Adding more indentation. commit 0a7c55705fcdb485b6a8278fe39dd31f6e21639e Author: Matt Gilman Date: Fri Mar 27 15:23:55 2015 -0400 NIFI-250: - Reloading newly and previously referenced controller services when modifying the referencing controller service. (ie adding/changing/removing the reference). commit 0d975028efdb07439ea21f1e3654b2c3cc939a93 Author: Matt Gilman Date: Fri Mar 27 13:14:19 2015 -0400 NIFI-250: - Fixing list item wrapping commit 1efa053b6af6b4d70bed608247b42bd9b92a9127 Merge: eaa8c51 92df0d7 Author: Mark Payne Date: Fri Mar 27 12:42:38 2015 -0400 Merge branch 'NIFI-250' of http://git-wip-us.apache.org/repos/asf/incubator-nifi into NIFI-250 commit eaa8c51f973b837ee5c41bb9fb7d09e98cd41824 Author: Mark Payne Date: Fri Mar 27 12:42:32 2015 -0400 NIFI-250: Fixed bug when restoring Reporting Tasks on NCM; added documentation to some services; fixed bugs in Standard Validators commit 92df0d7cc7074f4a14c571d1d69b2845b9a8ad7a Author: Matt Gilman Date: Fri Mar 27 09:39:27 2015 -0400 NIFI-250: - Reverting change to reload service and referencing components on done to do always. Since the enable/disable request may poll or be canceled by a user. commit be3254c947352486e5fe992184f8de6a5b20369f Author: Matt Gilman Date: Fri Mar 27 09:22:22 2015 -0400 NIFI-250: - Adding a check to ensure we don't attempt to reload a controller service that has been removed but is still referenced by another controller service. commit 21ab41fbe1cbd3460eb544ba96c18e120e7e67e3 Author: Matt Gilman Date: Fri Mar 27 09:16:24 2015 -0400 NIFI-250: - Fixing typo in the scope tooltip. commit 56f8dd972b4565a367fe317deee5826c215874bd Author: Matt Gilman Date: Fri Mar 27 08:30:52 2015 -0400 NIFI-250: - Ensuring the settings (controller service/reporting task) dialog is closed prior to opening a custom UI. commit 3eb1ac16a59738db683739e48d362260cf2c4e03 Author: Matt Gilman Date: Fri Mar 27 08:00:40 2015 -0400 NIFI-250: - Ensuring the style on the comments field is reset when changing its value. commit b51af3cb7a60fb11ddc2dedd328e35a92d39f0e3 Author: Matt Gilman Date: Thu Mar 26 15:12:09 2015 -0400 NIFI-250: - Fixing issues when clicking Add before selecting the type of controller service and reporting task to create. commit 5b56982d20b3a24ca59362a482c39ef6d4329b8d Author: Matt Gilman Date: Thu Mar 26 11:55:56 2015 -0400 NIFI-250: - Using the correct state variable when determining how to update a given controller service. commit 6c678bfe6adf9e15b294523603dbf230d731b3c9 Author: Matt Gilman Date: Thu Mar 26 10:42:24 2015 -0400 NIFI-250: - Prevented reading of the node response because when clustered the responses are merged and we need to use the updated entity. commit ffb4e6b68b57f23b41703613557f7bf0da5bbce5 Author: Matt Gilman Date: Thu Mar 26 09:13:23 2015 -0400 NIFI-250: - Only attempting to reload a controller service when one is actually referenced. commit c61a2afbb8b0696b883c06dfd92abb3f3b111bb6 Merge: 8a830be a5e140f Author: Mark Payne Date: Thu Mar 26 09:02:20 2015 -0400 Merge branch 'develop' into NIFI-250 Conflicts: nifi/nifi-api/src/main/java/org/apache/nifi/components/ValidationContext.java nifi/nifi-mock/src/main/java/org/apache/nifi/util/MockValidationContext.java nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/processor/StandardValidationContext.java commit 8a830bef412dfa44a54c840e24545defeed83f15 Author: Matt Gilman Date: Thu Mar 26 08:10:47 2015 -0400 NIFI-250: - Rendering reporting tasks in controller service referencing components. - Ensuring referenced controller services are reloading when a reporting task changes. - Automatically selecting reporting task tab and item in table when linking to one via controller service referencing components. commit 47e84c009db874b5f2151bd5eb92b403461c5730 Author: Matt Gilman Date: Thu Mar 26 08:04:17 2015 -0400 NIFI-250: - Fixing issue preventing reporting task from being included in controller service referencing components. commit fe24f368e8e317400d77491e904de6f76639fd53 Author: Matt Gilman Date: Thu Mar 26 07:04:51 2015 -0400 NIFI-250: - Handling the appropriate type of exception when creating a controller service. commit 4f01cd090972fd6b2712d914e7d2ea8eca8627b1 Merge: 8bef904 91e002e Author: Matt Gilman Date: Thu Mar 26 06:55:03 2015 -0400 Merge branch 'NIFI-250' of https://git-wip-us.apache.org/repos/asf/incubator-nifi into NIFI-250 commit 91e002eb90795a1c2d2711c3e20db520fce155ff Merge: 47435c8 5db358f Author: Mark Payne Date: Wed Mar 25 15:46:19 2015 -0400 Merge branch 'NIFI-250' of http://git-wip-us.apache.org/repos/asf/incubator-nifi into NIFI-250 commit 47435c8bb7207358dbfef2fcfdad5e3fdd054434 Author: Mark Payne Date: Wed Mar 25 15:46:09 2015 -0400 NIFI-250: Removed controller-services.xml and reporting-tasks.xml and associated properties commit 8bef904df75e1ca190075c5b78e6f858031ec4e8 Author: Matt Gilman Date: Wed Mar 25 15:29:18 2015 -0400 NIFI-250: - Including properties and descriptors in controller service referencing components. commit 5db358f4e580dc5a1129444ffc203a4041720160 Author: Matt Gilman Date: Wed Mar 25 15:27:35 2015 -0400 NIFI-250: - Populating the reporting task details when modifying a service that it references. commit 975db1a7b33383200b2045fd1c97e29216d1c2b3 Author: Matt Gilman Date: Wed Mar 25 15:07:30 2015 -0400 NIFI-250: - Fixing issue when reloading transitive controller service referencing components. - Allowing reporting tasks to be reloaded (when they reference controller services). commit 17b0fa7df170fa50f33236863481f008356cb1b4 Author: Matt Gilman Date: Wed Mar 25 15:03:39 2015 -0400 NIFI-250: - Addressing issues setting reporting task comments. - Including properties and descriptors in controller service referencing components. commit f71588065d788de4d3957593d88473c6bc13c2a1 Author: Matt Gilman Date: Wed Mar 25 11:07:13 2015 -0400 NIFI-250: - Reordering the invocation of the modal close handler and the hiding of the dialog. - Separating loading from showing the settings so the actions can be performed independently. This allows the transition back to the controller service/reporting task table more smooth when viewing usage or a custom UI. commit b8b9f2c2509d8101d2f749e4ce7c58ed7a52be96 Author: Matt Gilman Date: Wed Mar 25 09:57:37 2015 -0400 NIFI-250: - Ensuring requests are not attempted to be replicated if a custom UI provides an invalid component id. commit 2dccb5b4d98a194a7cee307b2252962eebdb67b1 Author: Matt Gilman Date: Wed Mar 25 09:28:55 2015 -0400 NIFI-250: - Adding better handling when a request is interpreted as a request to create a processor/controller service/reporting task and the type is not specified. commit e8294f29e2f5aef8cec0117cd09a2242b5ef2ca9 Author: Matt Gilman Date: Tue Mar 24 15:31:40 2015 -0400 NIFI-250: - Reopening the settings to the controller services/reporting tasks table after showing usage or custom UIs. commit d8a27993d0d7a7650f59d595c58eb99e5140c82c Author: Matt Gilman Date: Tue Mar 24 14:15:08 2015 -0400 NIFI-250: - Adding support to configure reporting task comments. Comments still need to be persisted to flow xml. commit ec21493622a9cc9ce0857c0b37d48a08f120a770 Author: Mark Payne Date: Tue Mar 24 13:07:05 2015 -0400 NIFI-250: Restore Comments and AnnotationData for Reporting Tasks on NCM; these were inadvertently skipped commit de21b6090c821a0dd4421d0637a2fd87a5a894d7 Merge: 4dcb9fd 953cb12 Author: Mark Payne Date: Tue Mar 24 12:49:48 2015 -0400 Merge branch 'NIFI-250' of http://git-wip-us.apache.org/repos/asf/incubator-nifi into NIFI-250 commit 953cb1227af70e846e47416ffdbc6eb3a1686d6a Author: Matt Gilman Date: Tue Mar 24 12:28:30 2015 -0400 NIFI-250: - Providing access to the merged node response when clustered. This is necessary as custom UIs will need to access to these details. - Ensuring the cluster processor endpoint is merged appropriately. commit 4dcb9fd761dfaec98790c41ca16344a1d0b89340 Author: Mark Payne Date: Tue Mar 24 10:16:47 2015 -0400 NIFI-278: Added documentation and ensured that all annotations invoke with consistent arguments commit 80b8c602433cc7b6589b417524a23b63c891d3f1 Author: Matt Gilman Date: Tue Mar 24 08:58:30 2015 -0400 NIFI-250: - Updating custom UI to fully support the new model (by removing references to processors). commit 691b4617d5568cd9a28f38d081b85d233ccbea44 Author: Matt Gilman Date: Tue Mar 24 08:57:34 2015 -0400 NIFI-250: - Updating custom UIs to support both old (deprecated) and new models. - Ensuring the annotation data is populated in outgoing DTOs. - Ensuring the component type is populated in the details for custom UIs. - Renaming the js module for custom UIs to exclude the reference to processors. commit dbaa219df12207d80b9e5f110443886b95bb5a7c Author: Matt Gilman Date: Mon Mar 23 14:32:28 2015 -0400 NIFI-250: - Renaming the standard nifi web configuration context. - Adding support for controller services and reporting tasks to the nifi web configuration context. commit 3e7ca3838039d724eba019214b073a8640e61149 Author: Matt Gilman Date: Mon Mar 23 13:09:15 2015 -0400 NIFI-250: - Continuing to merge changes from the content viewer with the newly supported component UI extensions. commit d3bb3ab829bbeaa93453509e33a7b3f165365987 Author: Matt Gilman Date: Mon Mar 23 11:34:18 2015 -0400 NIFI-250: - Only prompting users for the property name and immediately adding to the table. In the background, the property descriptor is loaded from the processor and then used when editing the property. commit 1c96bd08f7903f9acc845a542ccb7196f8a264c0 Author: Matt Gilman Date: Mon Mar 23 09:49:18 2015 -0400 NIFI-250: - Removing unneeded artifact. commit 1de514adf7a3dacc51823932b70de3f8f64c369a Author: Matt Gilman Date: Mon Mar 23 09:36:07 2015 -0400 NIFI-250: - More clean up post merge with the data viewer. commit d19471d6da17974ab6485f3653b4d793b30e895c Author: Matt Gilman Date: Mon Mar 23 08:20:04 2015 -0400 Resolving conflicts when merging develop into NIFI-250 commit 65e35e49a5bb1d81cf0933220bf1204131dc72d5 Merge: a4555d1 e05c9fd Author: Matt Gilman Date: Mon Mar 23 08:09:33 2015 -0400 Merge branch 'develop' into NIFI-250 Conflicts: nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-jetty/src/main/java/org/apache/nifi/web/server/JettyServer.java nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerFacade.java nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/pom.xml commit a4555d1a17252bedf503e443223670d34c5acd0d Merge: 6a31b94 fcff5c4 Author: Mark Payne Date: Fri Mar 20 15:52:18 2015 -0400 Merge branch 'NIFI-250' of http://git-wip-us.apache.org/repos/asf/incubator-nifi into NIFI-250 commit 6a31b94b39c07f943d9145889bc96078921cb2d0 Author: Mark Payne Date: Fri Mar 20 15:52:09 2015 -0400 NIFI-250: Fixed bug with templates related to services referencing other services commit fcff5c40a22a42bedbdd488dc540f4e80d6caf68 Author: Matt Gilman Date: Fri Mar 20 15:35:54 2015 -0400 NIFI-250: - Adding endpoints for obtaining [processor|controller service|reporting task] property descriptors. commit e0e2d161b5f577b2d3c012843c6fde2123cb3411 Author: Matt Gilman Date: Fri Mar 20 15:33:36 2015 -0400 NIFI-250: - Adding endpoints for obtaining [processor|controller service|reporting task] property descriptors. commit bd999d16ad804a0c71e6188dac0da57ceda58996 Author: Mark Payne Date: Fri Mar 20 14:51:26 2015 -0400 NIFI-250: Include referenced controller services in templates commit 02afcfbf4585eba10be1aeb90674c43933b63d88 Author: Matt Gilman Date: Fri Mar 20 14:39:15 2015 -0400 NIFI-250: - Removing link to jQuery migration script. commit 10ebbf0c3e9155e43dfbe866c42a27a021d3dc42 Author: Matt Gilman Date: Fri Mar 20 14:38:47 2015 -0400 NIFI-250: - Specifying the correct UI extension type before populate the service/task DTO. commit 07941b5209fa182b02ac7e2d66ebade0416c4319 Author: Matt Gilman Date: Fri Mar 20 14:38:09 2015 -0400 NIFI-250: - Closing the controller service and reporting task shell whenever a custom UI is opened. commit 5c98ccdbfb15d330ecce118feab98ac0681d6f1a Author: Matt Gilman Date: Fri Mar 20 13:09:38 2015 -0400 NIFI-250: - Renaming interface that custom UIs use to interact with the NiFi instance. - Javadocs. - Ported UpdateAttribute to use updated custom UI model. - Fixed bug when discovering types of components that support custom UIs. commit 24d787e5d40949659427e72d126dd2a225b6ff86 Author: Matt Gilman Date: Fri Mar 20 10:51:15 2015 -0400 NIFI-250: - Updating the way in which UI extensions interact with the underlying NiFi instance. commit ffa919a349906ef3a8fe178ab448fe69bf1bd4b6 Author: Mark Payne Date: Thu Mar 19 14:12:11 2015 -0400 NIFI-250: Fixed NPE commit c63a8c05ff64196a0fc77a2386c60bac88784ad0 Merge: 800f80b 42f8e81 Author: Mark Payne Date: Thu Mar 19 09:31:46 2015 -0400 NIFI-250: Fixed handling of controller services on startup and resolved merge conflicts commit 800f80bc05ead5f097c9e948f619e0dd681ca592 Author: Mark Payne Date: Thu Mar 19 09:28:05 2015 -0400 NIFI-250: Fixed controller service handling on startup commit 42f8e819811835c94744276cde741ebcdbb1a57e Author: Matt Gilman Date: Wed Mar 18 21:22:44 2015 -0400 NIFI-250: - Updating UI extension type. commit 6c058778534adae967ee34ec5fb2b8c20b66d43b Author: Matt Gilman Date: Wed Mar 18 16:09:44 2015 -0400 NIFI-250: - Starting to refactor support for UI extensions. - Deprecating previous support. commit f0c660c25ee0e2b1a6acd9b8e876144cad9b0d6b Merge: b45c09c e569af6 Author: Mark Payne Date: Wed Mar 18 10:12:13 2015 -0400 Merge branch 'develop' into NIFI-250 commit b45c09cacdbaf93f79dad95a9a258d82fed40831 Author: Mark Payne Date: Wed Mar 18 10:09:40 2015 -0400 NIFI-368: Implemented methods that were not implemented from interface commit 454c837ce1e63f780e648e86495c4e5c038fc090 Merge: d7210da e750579 Author: Mark Payne Date: Wed Mar 18 09:37:10 2015 -0400 Merge branch 'NIFI-250' of http://git-wip-us.apache.org/repos/asf/incubator-nifi into NIFI-250 commit d7210da40eda444267133fdfbbf2a8a378590e82 Author: Mark Payne Date: Wed Mar 18 09:37:06 2015 -0400 NIFI-250: Fix the way that services are restored at startup commit e750579d77ea3b69b438b57d5f468af6f51ff049 Author: Matt Gilman Date: Wed Mar 18 07:40:20 2015 -0400 NIFI-250: - Ensuring controller service comments are included when saving. - Adding a label to the comments in the service details dialog. commit f10bb997b84cce32d30ded918a93ea8f9d511016 Merge: 25ca4f8 c118ead Author: Matt Gilman Date: Wed Mar 18 06:58:44 2015 -0400 Merge branch 'develop' into NIFI-250 commit 25ca4f89e875bb8b7c89b75756793e1863190621 Merge: 769ed68 1cca300 Author: Matt Gilman Date: Tue Mar 17 16:03:08 2015 -0400 Merge branch 'develop' into NIFI-250 commit 769ed68c51233443bd210e6af3ee6c0b96a5de7d Author: Matt Gilman Date: Tue Mar 17 10:47:43 2015 -0400 NIFI-425: - Reseting the filter in the new controller service and new reporting task dialog. - Setting the initial focus. - Fixing the close handler configuration. commit 985aa134c4c5a43b1570b669e0d32c40be28e152 Author: Matt Gilman Date: Tue Mar 17 09:49:09 2015 -0400 NIFI-250: - Addressing cancel action when initiated by pressing escape. Previously it was canceling every dialog that was open when it should have only applied to the top most. commit bf7027c0760b433c3580a77282df24ed5aac67f2 Author: Matt Gilman Date: Tue Mar 17 08:32:13 2015 -0400 NIFI-425: - Filtering is not clear when closing the new processor dialog. commit aa21d073d55dff3702dbb811b80d0947f745538a Author: Matt Gilman Date: Tue Mar 17 08:18:06 2015 -0400 NIFI-250: - Merging develop into NIFI-250. commit c28d9f57287b5b95d6451a5365dbc2b3d4552422 Merge: 5a6723c 761e64a Author: Matt Gilman Date: Tue Mar 17 08:16:36 2015 -0400 NIFI-250: Merge develop into NIFI-250 commit 5a6723c0465d15e603ee7cbec9b1e58a80f0f036 Author: Matt Gilman Date: Mon Mar 16 15:56:14 2015 -0400 NIFI-250: - Fixing enable button label. commit 02b9e4b0a91e65625f10437bf9459e8005b3459c Author: Matt Gilman Date: Mon Mar 16 13:20:16 2015 -0400 NIFI-250: - Setting the correct visibility on the controller service name field. commit 24c86facff21f133a62bffe2f5927a45e1b9d059 Author: Matt Gilman Date: Mon Mar 16 12:34:17 2015 -0400 NIFI-250: - Using .html() instead of .text() as the service/task description is already escaped. commit dddf5e8841977321cce2f287be2728f3163ad5a4 Merge: 30f9323 eb757a4 Author: Matt Gilman Date: Mon Mar 16 11:41:26 2015 -0400 Merge branch 'develop' into NIFI-250 commit 30f9323a0cdf6f783c94ff0c7120b2c042dd8194 Author: Matt Gilman Date: Mon Mar 16 11:38:08 2015 -0400 NIFI-250: - Ensuring the new property dialog is only destroy when appropriate. There were cases when we wanted to clear the property table but leave the new property dialog intact. commit af855ba123db2dc8b9d5e96f47aabb5d76055130 Author: Matt Gilman Date: Mon Mar 16 10:19:28 2015 -0400 NIFI-250: - Using the correct element names when attempting to parse the reporting task configuration file. commit 5a57670c70012e5d8e091076f74c14b1fa78a735 Author: Matt Gilman Date: Mon Mar 16 09:21:21 2015 -0400 NIFI-250: - Ensuring the editable flag is set correctly when creating a property table. - Properly cleaning up the new property dialog. commit 19b6845b66da5e45d8a974f7d4d7cd01712ef698 Author: Matt Gilman Date: Mon Mar 16 08:43:55 2015 -0400 NIFI-250: - Ensuring the actions columns is always last. - Fixing field visibility (between read only and editable). commit 17b512ea8e720115b6bce1c5f65797d256392781 Author: Matt Gilman Date: Mon Mar 16 07:45:05 2015 -0400 NIFI-250: - Using a utility function for populating fields. - Using a label for scheduling strategy rather than the enum value. commit 92cbce496936c7e166c8666b379dcc8582b6d121 Author: Matt Gilman Date: Mon Mar 16 07:30:50 2015 -0400 NIFI-250: - Adjusting the layout of the reporting task dialog. - Renaming the reporting task run status column. commit 2c6f3a97276975899be6cc0266d6770bc88e1718 Merge: a1a7691 2de2134 Author: Matt Gilman Date: Mon Mar 16 07:04:11 2015 -0400 Merge branch 'develop' into NIFI-250 commit a1a769104881eaf44ca9736cd185edc972eb5807 Author: Matt Gilman Date: Fri Mar 13 14:48:08 2015 -0400 NIFI-250: - Updating the controller service dialog to support a read only and editable modes. commit 40670067cb67d9df8a1817fd0c5b3a86d627d3a9 Merge: b1d65e5 d122a83 Author: Matt Gilman Date: Fri Mar 13 14:17:05 2015 -0400 Merge branch 'NIFI-250' of https://git-wip-us.apache.org/repos/asf/incubator-nifi into NIFI-250 commit b1d65e560a00925345222508c370b91a878a9ab2 Author: Matt Gilman Date: Fri Mar 13 14:16:44 2015 -0400 NIFI-250: - Updating the controller service dialog to support a read only and editable modes. - Ensuring the new property dialog is cleaned up if a property table instance is re-initialized (dom leak). - Better rending controller service state. - Fixing issue with component history when opening the processor details dialog. commit d122a83633683285cc6fd6daf65f0f533a0da2ca Author: Mark Payne Date: Fri Mar 13 12:01:00 2015 -0400 NIFI-250: Fixed NPE commit 52ea335d7c7e75b01f604a13c15f2b30c1b8df5c Author: Matt Gilman Date: Fri Mar 13 11:07:31 2015 -0400 NIFI-250: - Only handling new controller service state if it's actually different than the current value. commit 50f0c123bcb8474b110aac0abc528c0f9b6fe24a Author: Matt Gilman Date: Fri Mar 13 10:46:49 2015 -0400 NIFI-250: - Including active threads in the reporting task dto. - Verifying actions taken against component referencing controller services. - Rendering the run state for reporting tasks (including active thread counts). commit cd69a423dd6411cdefe1a3131865fc45f440d426 Merge: 6b36aef f556490 Author: Matt Gilman Date: Fri Mar 13 10:11:22 2015 -0400 Merge branch 'NIFI-250' of https://git-wip-us.apache.org/repos/asf/incubator-nifi into NIFI-250 commit 6b36aefef942482a665182594fd3bb1044cad627 Author: Matt Gilman Date: Fri Mar 13 10:11:17 2015 -0400 NIFI-250: - Adding active thread count on reporting tasks. - Merging clustered responses for controller service and reporting task endpoints. commit f556490fb789706d4670ba6aef717de82285f29d Author: Mark Payne Date: Fri Mar 13 09:16:41 2015 -0400 NIFI-250: Added verify methods to controller service provider commit 79ddcb828e678e38ffb67f524bed03a72a3361dc Merge: d5a9a1a 7198912 Author: Matt Gilman Date: Fri Mar 13 06:49:10 2015 -0400 Merge branch 'develop' into NIFI-250 commit d5a9a1a625e28f3d0a7d6ed72019e84088a05794 Author: Matt Gilman Date: Thu Mar 12 12:36:17 2015 -0400 NIFI-250: - Updating button visibility to always show the enable/run buttons when the service/task is disabled/stopped. Previously the enable/run buttons were only visible the component did not have any validation errors. However, this caused the buttons to shift and the alignment to be off. commit 8cc58fe80bf402bb594e391f980e565f1a2318f3 Author: Matt Gilman Date: Thu Mar 12 12:11:33 2015 -0400 NIFI-250: - Adding a button to link to the usage of a controller service or reporting task. commit c9604a3fcacf4c33606ca0be767c9d89ef4424a0 Author: Matt Gilman Date: Thu Mar 12 11:10:39 2015 -0400 NIFI-250: - Adding support for configuring the scheduling strategy and scheduling period for reporting tasks. commit ca7b65262113a61ef9716dec0367fc24e19f5a05 Author: Matt Gilman Date: Thu Mar 12 08:16:13 2015 -0400 NIFI-250: - Trying to ensure accurate details when enabling/disabling controller service and minimize web requests. - Fixing issues with the enabled flag in the controller service and reporting task dialog. commit ae3c29aa562f15b8af89afdeec15475a2dd96fb5 Merge: 4f41202 56cc186 Author: Matt Gilman Date: Thu Mar 12 06:50:16 2015 -0400 Merge branch 'develop' into NIFI-250 commit 4f41202e9df732211f9a9b2ccd54d1f652984d8b Author: Matt Gilman Date: Wed Mar 11 16:36:56 2015 -0400 NIFI-250: - Updating controller service auditing to ensure transitive referenced components are reported correctly. - Restoring reference to reporting task provider. commit fe09680deaa76e9de7c962c98e50492173a1748e Merge: 4003542 85e38dc Author: Mark Payne Date: Wed Mar 11 15:48:04 2015 -0400 Merge branch 'NIFI-250' of http://git-wip-us.apache.org/repos/asf/incubator-nifi into NIFI-250 commit 40035429f886a501312b25b14f94cab38ed0c97f Author: Mark Payne Date: Wed Mar 11 15:47:58 2015 -0400 NIFI-250: Added missing methods to ReportingTaskProvider commit 85e38dc0555bf6a37ec378fc19b3a7811f8a32b3 Author: Matt Gilman Date: Wed Mar 11 15:40:23 2015 -0400 NIFI-250: - Setting the default name of the reporting task. commit 74d45aefbf6f74b04cd23808e4bb866cce027dc9 Author: Matt Gilman Date: Wed Mar 11 15:26:45 2015 -0400 NIFI-250: - Adding methods to obtain all reporting tasks. - Adding parameters to update the scheduled state of a reporting task. - Adding action buttons to the reporting task table. commit 2211741cad47ba24027f78f2e005cac888c8cb70 Author: Matt Gilman Date: Wed Mar 11 13:50:02 2015 -0400 NIFI-250: - Fixing issues showing the progress when enabling/disabling a controller service. - Ensure state for the service and its referencing components at all time since the action could be cancelled at any time. commit fc76a6165910441a372234dd3a4e626a83cbba48 Author: Mark Payne Date: Wed Mar 11 10:41:48 2015 -0400 NIFI-250: Fixed import that was accidentally removed commit 0d8cc34ae16979853a05b0ef214cd62694a31daf Merge: baa0e74 048c5d9 Author: Mark Payne Date: Wed Mar 11 08:45:36 2015 -0400 Merge branch 'NIFI-250' of http://git-wip-us.apache.org/repos/asf/incubator-nifi into NIFI-250 commit baa0e74cc37a6c74097da82ebf85d3b6e8e01d1a Author: Mark Payne Date: Wed Mar 11 08:45:33 2015 -0400 NIFI-250: Load controller services in correct order instead of arbitrary order commit 048c5d9aa757e3d2917d558857a59c4b6a6221f0 Author: Matt Gilman Date: Tue Mar 10 16:39:15 2015 -0400 NIFI-250: - Updating the progress label when enabling/disabling controller services and their referencing components. commit 51d70bff34174e19cd844404b8c0a61be79a8f39 Author: Matt Gilman Date: Tue Mar 10 14:32:30 2015 -0400 NIFI-250: - Showing the progress of enabling/disabling of a controller service so the user knows what the status of each step. This is especially important when the action fails someplace in the middle. commit 5cc6fda21b9081e426b8a65132d0d971ef645c15 Author: Matt Gilman Date: Mon Mar 9 16:38:39 2015 -0400 NIFI-250: - Updating the enable/disable dialog to show the current status while waiting for referencing components. commit 6d12f134ff22927d02c7c500a5650b855fa52289 Author: Matt Gilman Date: Mon Mar 9 14:57:18 2015 -0400 NIFI-250: - Adding a close button after enabling/disabling a controller service. This gives the user an opportunity to review issues prior to closing the dialog. commit eb6c5b5e23e5b0af4fdb88c6264ba0b05efdbb94 Merge: bde21cb 3533a4a Author: Matt Gilman Date: Mon Mar 9 14:14:07 2015 -0400 Merge branch 'develop' into NIFI-250 commit bde21cb8a6227e9b59f1af7266e949a5cd41b87f Author: Matt Gilman Date: Mon Mar 9 09:39:18 2015 -0400 NIFI-250: - Updating the layout of the enable/disable dialogs. - Showing referencing components validation errors where appropriate. commit 73459ccb5cfbc4aa81201e6476bb47ae450b49f7 Merge: 6f86758 342ca17 Author: Matt Gilman Date: Mon Mar 9 06:48:31 2015 -0400 Merge branch 'develop' into NIFI-250 commit 6f867585e1fade734dfecca434942d85fe20701d Author: Matt Gilman Date: Fri Mar 6 15:47:56 2015 -0500 NIFI-250: - Including validation errors in the controller service referencing components dto. commit b1dcab62a7f2ff6f2d15ef559058c7ec2358541e Author: Matt Gilman Date: Fri Mar 6 11:30:14 2015 -0500 NIFI-250: - Fixing missing import. commit 91f69b0cf23422838ca1f055064ae90e8749738f Author: Matt Gilman Date: Fri Mar 6 10:16:23 2015 -0500 NIFI-250: - Adding a factory bean for obtaining the appropriate ReportingTaskProvider given the configured operating mode (cluster/node). - Fixing copy/paste error with the ControllerServiceProvicer factory bean. commit ed22742c0f27c66205838237f312ddac01a8f61a Author: Matt Gilman Date: Fri Mar 6 10:14:26 2015 -0500 NIFI-250: - JS global hint. commit bc508cd905daefc08389e921df6c681fa67dafdf Author: Matt Gilman Date: Fri Mar 6 09:42:48 2015 -0500 NIFI-250: - Reseting the table size when opening the new controller service/reporting task dialog. - Fixing typo during scroll events over the breadcrumbs. - JS global hint. commit e5a314116bb43c54c2a920a7930423de038004cc Merge: 6b429bf bd74063 Author: Matt Gilman Date: Fri Mar 6 08:19:37 2015 -0500 Merge branch 'develop' into NIFI-250 commit 6b429bf42b5f31e6b7bfc18d932c9cd48a67e702 Merge: f34e343 883c4ac Author: Matt Gilman Date: Fri Mar 6 08:19:26 2015 -0500 Merge branch 'NIFI-250' of https://git-wip-us.apache.org/repos/asf/incubator-nifi into NIFI-250 commit 883c4ac9d7a1cdbd6989fb1312624a752c490461 Merge: 83eff8d 2d4aebf Author: Mark Payne Date: Wed Mar 4 15:50:58 2015 -0500 Merge branch 'NIFI-250' of http://git-wip-us.apache.org/repos/asf/incubator-nifi into NIFI-250 commit 83eff8d6df7b485a4d0eb03f176e4a28ff81df3b Author: Mark Payne Date: Wed Mar 4 15:50:30 2015 -0500 NIFI-250: Creating ReportingTaskProvider commit f34e34391716c0168b74241c284db3ffab444fe5 Merge: 2d4aebf 5e0026c Author: Matt Gilman Date: Wed Mar 4 09:30:23 2015 -0500 Merge branch 'develop' into NIFI-250 commit 2d4aebf33b50b7603f2c19ba63d368bf125ce498 Author: Matt Gilman Date: Fri Feb 20 15:08:39 2015 -0500 NIFI-250: - Continuing to setup Reporting Task management in the UI. commit c1077baf95f8c7e5ffc16ccea190b56af59c3fcc Author: Matt Gilman Date: Fri Feb 20 13:39:11 2015 -0500 NIFI-250: - Enabling/Disabling controller service and referencing components in the appropriate order. - Continuing to setup Reporting Task management in the UI. commit 573a8aa353bb47cbfc782d06786f31a93faa3d7a Author: Matt Gilman Date: Thu Feb 19 16:30:50 2015 -0500 NIFI-250: - Using controller service state field instead of enable flag. - Continuing to setup Reporting Tasks. commit a227fe46b542fdcd6dd6fe57f11d678da237bc98 Author: Mark Payne Date: Thu Feb 19 15:56:38 2015 -0500 NIFI-250: Fixed bug that occurred in refactoring commit be00d0caf34cb2ec4f192563cb2e8707d145e8b1 Author: Mark Payne Date: Thu Feb 19 15:20:24 2015 -0500 NIFI-250: Added overloaded constructor to MockControllerServiceINitializationContext that allows no logger to be provided and autocreates one commit f246565f7e5344db91f731e003c7e35e1d7cbe95 Author: Mark Payne Date: Thu Feb 19 15:04:13 2015 -0500 NIFI-250: Incorporate new logic for controller service state commit 852cc607f6fad61178f7c9d3a26807c24eee4e0e Author: Mark Payne Date: Thu Feb 19 15:03:54 2015 -0500 NIFI-368: generate bulletins when there are problems communicating with reporting tasks and controller services commit ee63a10b8654c7a635e359eba9a7531aaf70fe6d Author: Matt Gilman Date: Thu Feb 19 14:55:11 2015 -0500 NIFI-250: - Removing enabled flag from controller service DTOs. commit 767f37b8311f5fbbcfa5aacd6ef1230103034f09 Author: Mark Payne Date: Thu Feb 19 14:46:32 2015 -0500 NIFI-250: Updated javadocs to clarify how lifecycle annotations are used; cleaned up handling of exceptions in scheduler commit 5f2a4358869853deff7747ae3908d6952ee94001 Author: Mark Payne Date: Thu Feb 19 14:45:46 2015 -0500 NIFI-368: added componentlog to the initialization contexts for appropriate components and made abstract components return logger via getLogger() methods commit 8f78d619751c2bdb23f6c192a92716517922846a Author: Mark Payne Date: Thu Feb 19 14:29:21 2015 -0500 NIFI-368: Extracted methods from ProcessorLog out to ComponentLog and made ProcessorLog extend from ComponentLog. This maintains backward compatibility while providing more generic loggers for other components commit 7de30ab15ad9570233c4bff68f37acf324a66dda Author: Mark Payne Date: Thu Feb 19 13:30:37 2015 -0500 NIFI-250: Refactoring of controller service states commit 81d84546e7b456ae55cefe8892ddfb71a729bac9 Merge: 2ac6432 fac6cd7 Author: Mark Payne Date: Thu Feb 19 10:38:22 2015 -0500 Merge branch 'NIFI-250' of http://git-wip-us.apache.org/repos/asf/incubator-nifi into NIFI-250 commit fac6cd7ac6dc1ed883a18369ebfbbde0886b11ec Merge: a3836d8 57b5d58 Author: Matt Gilman Date: Thu Feb 19 07:50:29 2015 -0500 Merge branch 'develop' into NIFI-250 commit a3836d832d4cdc75a6b9317a04fed00c9b062c22 Author: Matt Gilman Date: Wed Feb 18 15:17:48 2015 -0500 NIFI-250: - Starting to add an endpoint for Reporting Tasks. commit ffa7fd06d1c85e8aa45568e27f155309fb9bede9 Author: Matt Gilman Date: Wed Feb 18 13:38:30 2015 -0500 NIFI-250: - Adding the scope to the disable controller service dialog. commit bbe185609ad037b8f1ed6529e25bea1b60613653 Author: Matt Gilman Date: Wed Feb 18 13:38:11 2015 -0500 NIFI-250: - Updating the controller services references endpoint to accept either an enabled flag for referencing controller services or a state for referencing schedulable components. commit 4efd9622109f584daddb76d6b5de2399bd7d8209 Author: Matt Gilman Date: Wed Feb 18 09:49:34 2015 -0500 NIFI-250: - Updating the polling for controller service status when disabling. commit 2ac643292c691d451cf98e2a309bff72182b0a88 Merge: d30a184 6457929 Author: Mark Payne Date: Wed Feb 18 08:15:45 2015 -0500 Merge branch 'NIFI-250' of http://git-wip-us.apache.org/repos/asf/incubator-nifi into NIFI-250 commit 6457929b25ba5923bebeb6ab4566e4208d1a092e Author: Matt Gilman Date: Tue Feb 17 15:32:31 2015 -0500 NIFI-250: - Adding auditing to activating/deactivating controller service references. commit e61a01ac6968ba2f1e4f8fa1d3ca9c2cbca400e6 Author: Matt Gilman Date: Tue Feb 17 14:55:44 2015 -0500 NIFI-250: - Only showing the enable button when the controller service is valid. - Adding polling during deactivation of controller service references to know when it's ok to attempt to disable the controller service. commit f22407f59347010aede5c12cd64652a381f4fd59 Author: Matt Gilman Date: Tue Feb 17 14:54:32 2015 -0500 NIFI-250: - Updating the controller service references endpoint to accept an activated flag that drivens whether processors/reporting tasks are started/stopped and controller services enabled/disabled. commit e4e61daa26c996d3234982cf1409bb501631b9d0 Author: Matt Gilman Date: Tue Feb 17 12:48:39 2015 -0500 NIFI-250: - Updating fill color dialog styling for use in updated modal plugin. - Making the fill color dialog draggable. commit 4ce7202d93551acaea8e6f782df694eee1435ec5 Merge: 2769db7 0047fa4 Author: Matt Gilman Date: Tue Feb 17 12:36:47 2015 -0500 Merge branch 'develop' into NIFI-250 Conflicts: nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/css/dialog.css commit 2769db7b4e783540b03bf2fee800a56d86e79d4d Author: Matt Gilman Date: Wed Feb 11 16:17:03 2015 -0500 NIFI-250: - Adding an enable controller service dialog. commit d30a1843c24caccf822811649332dad7e1aa5b17 Author: Mark Payne Date: Wed Feb 11 16:08:20 2015 -0500 NIFI-250: Deleted dead code commit 62b0669e121e817cd80ba21b8104ad9e210ea531 Author: Matt Gilman Date: Wed Feb 11 11:25:04 2015 -0500 NIFI-250: - Renaming references to referencing components to be more accurate. - Fixing borders around the referencing components. commit 819c76b885268db294b62e11095faf52eaebf57c Merge: ceede01 371e010 Author: Matt Gilman Date: Wed Feb 11 11:14:37 2015 -0500 Merge branch 'NIFI-250' of https://git-wip-us.apache.org/repos/asf/incubator-nifi into NIFI-250 commit 371e0100d35864f0092b133a7decac34ffb4be33 Author: Mark Payne Date: Wed Feb 11 11:11:49 2015 -0500 NIFI-250: Move activate/deactive methods for controller services' referencing components to ControllerServiceProvider commit ceede01b8ffe57c003ac246b51854dd6b01ad70e Author: Matt Gilman Date: Wed Feb 11 10:45:14 2015 -0500 NIFI-250: - Adding endpoints for retrieving and updating controller service referencing components. commit 8a414b3ca9a542170cf31d6396c3e3e2947b90fc Author: Matt Gilman Date: Wed Feb 11 10:43:37 2015 -0500 NIFI-250: - Adding endpoints for retrieving and updating controller service referencing components. commit ed4d22c1ff17395a683a83d9f57dfdc842ce175f Author: Matt Gilman Date: Wed Feb 11 09:22:18 2015 -0500 NIFI-250: - Additionally refactoring of removal of ControllerServiceProvider usage. commit 0d2041b05ffc618c2393713e96c44cf8384aa6e3 Author: Matt Gilman Date: Wed Feb 11 08:54:19 2015 -0500 NIFI-250: - Replacing usage of ControllerServiceProvider with ControllerServiceLookup. commit 3d38d8c98c180f5446b0f2dafea78eeec38e432e Merge: 4bc5ed1 56a6bc4 Author: Matt Gilman Date: Wed Feb 11 07:09:05 2015 -0500 Merge branch 'develop' into NIFI-250 commit 4bc5ed13ef981b49390b1e8460105e8ed227b446 Author: Matt Gilman Date: Tue Feb 10 14:34:48 2015 -0500 NIFI-250: - Rendering the references in the disable controller service dialog. - Reloading controller service table when appropriate. - Padding in dialogs with a border. commit 5390c7626a99c076daf05d06118815f902ab7b51 Author: Mark Payne Date: Tue Feb 10 11:54:17 2015 -0500 NIFI-250: Fixed NPE in StandardSSLContextService commit c25a2caecd31638d3564ea4892986d0580234783 Author: Mark Payne Date: Tue Feb 10 11:53:51 2015 -0500 NIFI-250: Persist Controller Services and Reporting Tasks on NCM commit 33f551630ac2664cf938834c7e450d04da7ec9f0 Author: Mark Payne Date: Tue Feb 10 11:08:05 2015 -0500 NIFI-250: Remove references to Controller Service when the referencing component is removed commit ea8cb59848e5cc75c201ce1b9521aa56eb293674 Author: Mark Payne Date: Tue Feb 10 09:48:01 2015 -0500 NIFI-250: Expose number of active threads in ProcessorNode, ReportingTaskNode commit 40e25066742f910a2c34881eb216a18d5233cdf1 Author: Matt Gilman Date: Mon Feb 9 14:56:22 2015 -0500 NIFI-250: - Reloading components that reference controller services when appropriate. - Created an icon for enabling a controller service. commit 4481213849cee69537c23d54871c4dc07bd7fb87 Author: Matt Gilman Date: Mon Feb 9 13:28:20 2015 -0500 NIFI-250: - Updating dialogs to address issues when showing the borders within the modal plugin (especially in the controller service configuration dialog). - Fixing availability when creating a controller service in standalone. commit 94d112b6ec90cc9df4adc9b85cc9df45908a1f36 Merge: c7ee77b b64fe47 Author: Matt Gilman Date: Mon Feb 9 09:48:23 2015 -0500 Merge branch 'develop' into NIFI-250 commit c7ee77be9781057109021c9db6981d9905397b3e Merge: 7da7c93 0133f84 Author: Matt Gilman Date: Mon Feb 9 07:35:46 2015 -0500 Merge branch 'develop' into NIFI-250 commit 7da7c9327029a84581c5aff5f3435f2706d2c18e Author: Matt Gilman Date: Fri Feb 6 15:32:09 2015 -0500 NIFI-250: - Updating the UI to gather the availability for a controller service during creation. - Removing the concept of availability from the core as the service/task can only be available on the cluster manager OR nodes. - Updating web tier to support this concept. commit 55949a428f45b7e615c09d6f870196fcc95acfe7 Author: Mark Payne Date: Fri Feb 6 10:45:59 2015 -0500 NIFI-250: Removed 'Availability' concept from reporting tasks and controller services because it is implicitly, depending on if on node or ncm commit d95ca18a0cc6834defbf184e2cae5ee93228e832 Author: Matt Gilman Date: Fri Feb 6 08:55:07 2015 -0500 NIFI-250: - Javadocs. commit 6892f19d61ebd4ae227b43452ba9c4303bd522eb Author: Matt Gilman Date: Fri Feb 6 07:52:38 2015 -0500 NIFI-250: - Including the user who last modified the flow in the refresh tooltip. commit 9f9466cb418b01295ecbe727dca07e82ddda9cb5 Merge: 4a8da60 d36a71c Author: Matt Gilman Date: Fri Feb 6 07:17:11 2015 -0500 Merge branch 'develop' into NIFI-250 commit 4a8da60334f4b78a123906a6a712d95ec873c687 Author: Matt Gilman Date: Thu Feb 5 15:59:35 2015 -0500 NIFI-250: - Refactoring revision checking so that we can lock appropriately on the Cluster Manager to manage controller services running there while other concurrent requests can be replicated amongst the cluster. commit 17add531f656b6d8065bed74a887acaa00a07f9a Merge: 3425dee 4a49ba7 Author: Matt Gilman Date: Thu Feb 5 07:13:56 2015 -0500 Merge branch 'develop' into NIFI-250 commit 3425dee1f2aca679235b434b40bb49be8099ff55 Author: Matt Gilman Date: Wed Feb 4 14:04:38 2015 -0500 NIFI-250: - Invoking the appropriate save when updating controller services. commit 6b91546d9d04dcb127cada3cdc907e9ef98abf42 Author: Matt Gilman Date: Wed Feb 4 13:13:09 2015 -0500 NIFI-250: - Fixing use of the optimistic locking manager factory bean. commit cc8b096ffa11ee44f6d5fb5f07de51826dd27b20 Merge: 13fb1a7 ed53b46 Author: Matt Gilman Date: Wed Feb 4 12:44:56 2015 -0500 Merge branch 'develop' into NIFI-250 Conflicts: nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/history/nf-history-table.js nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/nf-common.js nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PostHTTP.java commit 13fb1a758b59cd7e205caed694f28b4d0dba9d82 Merge: 22822d3 037f36d Author: Matt Gilman Date: Mon Feb 2 09:27:26 2015 -0500 Merge branch 'develop' into NIFI-250 commit 22822d33a2159d1f9a2b12caafbce7c2b46ebe52 Author: Matt Gilman Date: Mon Feb 2 09:10:12 2015 -0500 NIFI-250: - Updating the ControllerService endpoint to specify the availability of the service. - Updating the optimistic locking manager to ensure the proper revision is checked. commit 346cc0cf14aad3f2240336eda711e63f29e62a7c Author: Matt Gilman Date: Mon Feb 2 07:55:57 2015 -0500 NIFI-250: - Using the ControllerServiceProvider instead of delegating directly to the FlowController. This will allow us to work with the NCM when clustered. commit 0cb1adbc1f1bf0272be813bf58b90444243b65b5 Merge: d1caa7e cc25d1d Author: Matt Gilman Date: Mon Feb 2 07:52:38 2015 -0500 Merge branch 'NIFI-250' of https://git-wip-us.apache.org/repos/asf/incubator-nifi into NIFI-250 commit d1caa7ee3e2e8662eeb2b368b2ed81fface22711 Author: Matt Gilman Date: Mon Feb 2 07:52:32 2015 -0500 NIFI-250: - Updating REST API for managing controller services. - Starting to add a dialog for disabling a controller service. commit cc25d1d21cae8b30eff0db66d1ad9b6bbae5351d Author: Mark Payne Date: Fri Jan 30 14:43:54 2015 -0500 NIFI-250: Updated to pass Availability when creating controller services commit e056bb7d70de8e1dea7bab9f59d9f466aef45eda Author: Mark Payne Date: Fri Jan 30 14:37:42 2015 -0500 NIFI-250: Fixed bug that caused EOFException if no reporting tasks defined on NCM commit 4ae2a10e6d130353ea34aa403d3bcbad7a2ab4e8 Author: Mark Payne Date: Fri Jan 30 13:01:00 2015 -0500 Load Controller Services from flow.tar file instead and external file commit 52149d8510d8c517063bfd6359bde3950862db77 Author: Mark Payne Date: Fri Jan 30 12:44:30 2015 -0500 NIFI-250: Change Availability to just NODE or NCM; update NCM to store controller services and reporting tasks in separate entries in tar file instead of in flow.xml commit 3102e08378b34cb869c708faa05c427de6b68139 Author: Mark Payne Date: Fri Jan 30 10:18:54 2015 -0500 NIFI-250: Updated ControllerServiceProvider to have a 'createControllerService' that takes no identifier and randomly generates one commit 3344cef3365dfc620f11aaf18fcfc8ba7d58e16a Author: Mark Payne Date: Fri Jan 30 09:05:58 2015 -0500 NIFI-250: Updated controller services to use appropriate defaults and use .identifiesControllerService instead of using the old way of obtaining controller services; do not fail to startup if controller service is invalid commit 1682e47aa3820bca6ec80e6009a16d0572fb1a9b Merge: fbc14e0 e7beef8 Author: Mark Payne Date: Fri Jan 30 08:36:55 2015 -0500 Merge branch 'NIFI-250' of http://git-wip-us.apache.org/repos/asf/incubator-nifi into NIFI-250 commit 2de754a11af2f3d545ae9358c06dd6838e3d920c Author: Matt Gilman Date: Fri Jan 30 08:35:18 2015 -0500 NIFI-250: - Showing controller service validation errors. commit 600f8f7869161b50dc9939401ce8a91a64e3462a Author: Matt Gilman Date: Fri Jan 30 07:46:47 2015 -0500 NIFI-250: - Adding support to go to a controller service. commit e7beef8d224797c057650787c4041b15dbef87af Merge: 1ebaf1d bafa945 Author: Matt Gilman Date: Fri Jan 30 07:22:25 2015 -0500 Merge branch 'develop' into NIFI-250 commit fbc14e0a3a46c0511d6135e86797cfc14c81d7d5 Merge: 9a6acab 1ebaf1d Author: Mark Payne Date: Thu Jan 29 16:26:08 2015 -0500 Merge branch 'NIFI-250' of http://git-wip-us.apache.org/repos/asf/incubator-nifi into NIFI-250 commit 1ebaf1d29c7294b07f68def9d463c791f6d68cf9 Author: Matt Gilman Date: Thu Jan 29 13:06:54 2015 -0500 NIFI-250: - Addressing the border visibility when toggling a reference block. commit b98a7220c04582ad7305429ddee850096c2139cd Author: Matt Gilman Date: Thu Jan 29 12:56:15 2015 -0500 NIFI-250: - Showing components that reference a given controller service. commit 9a6acab3734fbbb670fb5db2388ac85ef9b6fc6d Author: Mark Payne Date: Thu Jan 29 11:24:56 2015 -0500 NIFI-250: Do not try to load controller services and reporting tasks from old .xml files because they are now in flow.xml commit 60ad998153f6c391f4a36f25b91629bde7d3e98b Author: Mark Payne Date: Thu Jan 29 11:24:11 2015 -0500 NIFI-250: Load Controller Services before processors or reporting tasks on restart commit cfaafa3c291bbbe1009c10c10204befea0482bbe Merge: 52d329c 4737a41 Author: Mark Payne Date: Thu Jan 29 10:55:09 2015 -0500 Merge branch 'NIFI-250' of http://git-wip-us.apache.org/repos/asf/incubator-nifi into NIFI-250 commit 4737a41a9ce7de540c7dce1d19e54802ce9127a9 Author: Matt Gilman Date: Thu Jan 29 10:51:18 2015 -0500 NIFI-250: - Addressing some issues with reloading controller services on restart. - Starting to add the controller service references to the dialog. - Reloading processors and controller services before configuration to ensure their property descriptors are current. commit 52d329ca036e059f50a169b726f4c202859ed0e1 Author: Mark Payne Date: Thu Jan 29 10:02:15 2015 -0500 NIFI-250: Restore appropriate reporting task id on restart commit 59f5b95c3ac3b1660da35aec63c22b23347a5aa3 Author: Mark Payne Date: Thu Jan 29 09:56:55 2015 -0500 NIFI-250: Restore Controller Service using appropriate ID on restart commit d60710b6fbbddc8bb1b6d8f5a7b078bbf7e4bd0a Merge: f18aca1 bb628b0 Author: Mark Payne Date: Thu Jan 29 09:05:57 2015 -0500 Merge branch 'NIFI-250' of http://git-wip-us.apache.org/repos/asf/incubator-nifi into NIFI-250 commit f18aca1e0a6a701b9390cbe80e131296fbcbc7e7 Author: Mark Payne Date: Thu Jan 29 09:05:43 2015 -0500 NIFI-250: Fixed bug where validator returns controller service identifier instead of name in description. Fixed bug where controller serivce references are not updated when property is removed from processor/service/reporting task commit 31caadb20a85c2afa53969b1bd60f1979e1f004f Author: Mark Payne Date: Thu Jan 29 08:49:44 2015 -0500 NIFI-250: Fixed bug that caused IllegalStateException if property not set on controller service or reporting task commit bb628b07a905ddb7b6dbf0c7c216d00627f82668 Author: Matt Gilman Date: Wed Jan 28 15:55:00 2015 -0500 NIFI-250: - Adding support for showing property history for controller services. commit 0d1f80f22ecae9d8c5cc63fb7224ac551c0571d2 Author: Matt Gilman Date: Wed Jan 28 15:54:26 2015 -0500 NIFI-250: - Adding support for showing property history for controller services. commit 5f571be29a4d934ec65701093dcc67964664b3f7 Author: Matt Gilman Date: Wed Jan 28 15:02:02 2015 -0500 NIFI-250: - Allowing read only users to view the controller settings (including controller services and reporting tasks). commit 4564a137cbd929b7620c1008a1fc14fa60680421 Author: Matt Gilman Date: Wed Jan 28 14:28:47 2015 -0500 NIFI-250: - Showing refresh notice in the settings tab when the flow changes externally (another user). commit c9d9d3199a40b602f1dfb2ccc532c264b4e223ab Author: Matt Gilman Date: Wed Jan 28 13:13:36 2015 -0500 NIFI-250: - Auditing controller service actions. commit 0aee34c287c28a01022f23a8a5809de489282734 Author: Matt Gilman Date: Wed Jan 28 12:40:02 2015 -0500 NIFI-250: - Auditing controller service actions. commit d90e9d6a7203db4e612ca27fb2567dd2b92c8d88 Author: Matt Gilman Date: Wed Jan 28 10:51:36 2015 -0500 NIFI-250: - Documentation. - Prepping the list of controller service references. commit 6581eb1360902225312d532c07b86343df175bc5 Author: Matt Gilman Date: Wed Jan 28 10:32:15 2015 -0500 NIFI-250: - Including the controller service references. commit 68583ab90a50f5929859144326dc105cc4753acd Author: Matt Gilman Date: Wed Jan 28 09:36:34 2015 -0500 NIFI-250: - Fixing the NODE_ONLY value. commit e06e423ff191696f0ab2a081434ac7b26e183b91 Author: Matt Gilman Date: Wed Jan 28 09:30:33 2015 -0500 NIFI-250: - Only showing the availability field when clustered. commit 91aa952f017dac0bf8cb1597ec3440f9bd8cda7a Author: Matt Gilman Date: Wed Jan 28 09:24:27 2015 -0500 NIFI-250: - Adding support to enable/disable controller services through the actions in the table. commit 5d5cb8f106c630e92852cf0fdefdad1c602c4520 Author: Matt Gilman Date: Wed Jan 28 08:35:17 2015 -0500 NIFI-250: - Wiring up updating and deleting controller services. - Ensuring the new property dialog is closed when clearing the grid. commit 2303570eb68da63ebbd3a438555e3f523ec7308d Author: Matt Gilman Date: Wed Jan 28 08:33:42 2015 -0500 NIFI-250: - Adding parameters to the update controller service endpoint. commit 593afddaf0d1a0746064ad0cf3b74a7b06dc7031 Author: Matt Gilman Date: Tue Jan 27 16:21:41 2015 -0500 NIFI-250: - Documentation. - Updating how actions are triggered in the controller service table. commit cf188ebc838ba3f0c26ae14c30e73b8a33510735 Author: Matt Gilman Date: Tue Jan 27 16:20:35 2015 -0500 NIFI-250: - Starting to implement the controller service DAO. commit 526e18d00a1bdf8a63ebc70fd257ea23bf678425 Author: Matt Gilman Date: Tue Jan 27 15:20:01 2015 -0500 NIFI-250: - Using the property table plugin in the read only processor details dialog. - Clean up. commit 2d7c700d173e757ca51b7b9832c9b385a67061c5 Merge: ec082f1 ea17dbe Author: Matt Gilman Date: Tue Jan 27 13:53:02 2015 -0500 Merge branch 'NIFI-250' of https://git-wip-us.apache.org/repos/asf/incubator-nifi into NIFI-250 commit ec082f1eead6b37e3b26e579f958cc0cde154d22 Author: Matt Gilman Date: Tue Jan 27 13:52:42 2015 -0500 NIFI-250: - Creating a jQuery plugin for rendering a table of properties (since we now want to configure processors, controller services, and reporting tasks). commit 5785198f66c2d3d2aa17767a8941f194afc5638b Author: Mark Payne Date: Tue Jan 27 09:56:50 2015 -0500 NIFI-250: Removed TODO comment that was completed commit ea17dbec6e1b4384c91fab9e54e30b29c6190c8d Author: Mark Payne Date: Tue Jan 27 09:48:45 2015 -0500 NIFI-250: Renamed 'getComment' and 'setComment' in ControllerServiceNode and ReportingTaskNode to 'getComments' and 'setComments' to be more consistent with how naming throughout the rest of the app commit f14a34f6686621501ddcc68efbcc64f001c96ab8 Author: Mark Payne Date: Tue Jan 27 09:22:40 2015 -0500 NIFI-250: Updated flow.xml schema to take reporting tasks and controller services into account; removed schemas for reporting tasks and controller services. commit 24b4f1cf1109af53d6fd94d81fe81141e7dd0423 Merge: 4de0fd0 cb84829 Author: Mark Payne Date: Tue Jan 27 08:36:57 2015 -0500 Merge branch 'NIFI-250' of http://git-wip-us.apache.org/repos/asf/incubator-nifi into NIFI-250 commit cb84829b31cd709cedf866ac41854c3a2e74a158 Author: Matt Gilman Date: Tue Jan 27 08:29:50 2015 -0500 NIFI-250: - Renaming comment to comments. - Ensuring type is included in DTO and create request. commit 4de0fd02678ce13d5a8b08edbc35a677560ded4d Author: Mark Payne Date: Mon Jan 26 19:05:31 2015 -0500 NIFI-250: Only run controller services and reporting tasks on the nodes/ncm according to their Availability commit 2df4500c05ac91d46614e9ca41974fed890d2646 Author: Mark Payne Date: Mon Jan 26 15:17:04 2015 -0500 NIFI-250: Implemented fingerprinting and reloading of controller services and reporting tasks from flow.xml commit 86d15f9e1c3b28a9ae587a1cb0c73cd2b6e8b51c Author: Mark Payne Date: Mon Jan 26 13:55:55 2015 -0500 NIFI-250: Serialize and deserialize controller services and reporting tasks in the flow.xml commit ff43b039085c56bb5f53148e25878ad71972a3db Merge: 7a3d208 2da5792 Author: Mark Payne Date: Mon Jan 26 11:53:17 2015 -0500 NIFI-250: Merged changes commit 7a3d208f031122ab13349ace8aeafac9fe5c54ac Author: Mark Payne Date: Mon Jan 26 11:42:01 2015 -0500 NIFI-250: Allow Controller Services to be created without id and properties; allow controller service lookup to provide name of controller service given an id commit 80f02e47256590c57a811b79040b70c827e71ff2 Author: Mark Payne Date: Mon Jan 26 10:31:06 2015 -0500 Added Eclipse-specific files to .gitignore commit 2da57924286fe1f9ae320e270ba0aba77a08d8b7 Author: Matt Gilman Date: Mon Jan 26 09:39:14 2015 -0500 NIFI-250: - Ensuring the tables are resized when appropriate. commit 1854ebed22387d0f1a82bd9c52ab1ab5ae5c7e21 Author: Matt Gilman Date: Mon Jan 26 09:38:59 2015 -0500 NIFI-250: - Fixing NPE in stubbed out method. commit 3e77e7d32e2c4e39a8247eacb44681a57bc3b618 Author: Matt Gilman Date: Mon Jan 26 09:29:34 2015 -0500 NIFI-250: - Supporting updated API. commit e989215fa539f61eba7c915d1b04adebcc771c69 Merge: 0c021c7 6b560b9 Author: Matt Gilman Date: Mon Jan 26 09:19:12 2015 -0500 Merge branch 'develop' into NIFI-250 commit 0c021c782589b4fe439dcb0f81d229fde6cd9327 Author: Matt Gilman Date: Mon Jan 26 09:18:06 2015 -0500 NIFI-250: - Starting to load the controller services and prepare for adding them. commit 35616e9ad64cd2985cf2e2cd1a2aa5fce33d980d Merge: 6622317 33cee9d Author: Matt Gilman Date: Mon Jan 26 07:15:00 2015 -0500 Merge branch 'develop' into NIFI-250 commit 6622317c1bcfb6b0a8f15cd1889fd644b2264a3a Author: Matt Gilman Date: Fri Jan 23 10:54:34 2015 -0500 NIFI-250: - Adding a field to collect the controller service name during creation. commit cb4c654c7f244ee7222fb444ccca096b8f56e164 Merge: e604caf 43e646e Author: Matt Gilman Date: Fri Jan 23 09:30:27 2015 -0500 Merge branch 'develop' into NIFI-250 commit e604cafe79f5a90e9179a9f2e46e3cbbb7f47380 Author: Matt Gilman Date: Thu Jan 22 15:24:17 2015 -0500 NIFI-250: - Adding an endpoint to manage ControllerServices. commit c8ea6830672350e1469a21eeda2ce02a6f6236ac Author: Matt Gilman Date: Wed Jan 21 08:03:15 2015 -0500 NIFI-250: - Moving files into the appropriate renamed directories. commit f2b9f2c8d418d14fd982d721ccbb6b2bb0874de1 Author: Matt Gilman Date: Wed Jan 21 08:02:50 2015 -0500 NIFI-250: - Moving files into the appropriate renamed directories. commit c5d452c1435d3d32df3da4100ca740ca372adb3d Merge: e8d2bbf cff8b0d Author: Matt Gilman Date: Wed Jan 21 07:37:09 2015 -0500 Merge branch 'develop' into NIFI-250 commit e8d2bbfe3c286fc21dea04e993771985d843b6e6 Author: Matt Gilman Date: Tue Jan 20 13:35:18 2015 -0500 NIFI-250: - Making the new controller services table collapsible. - Addressing issues with filtering, rollup, tags, and collapsing. commit 4836385628b5622aa77c8adacd3a5241f8ee1829 Author: Matt Gilman Date: Tue Jan 20 12:28:41 2015 -0500 NIFI-250: - Ensuring the entire hierarchy is returned for each service type. commit 66bd61e07bff17ddf7d672088e13d0b01bbfc96f Author: Matt Gilman Date: Tue Jan 20 08:40:37 2015 -0500 Merge branch 'NIFI-250' of https://git-wip-us.apache.org/repos/asf/incubator-nifi into NIFI-250 Conflicts: nifi/nar-bundles/framework-bundle/framework/web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/DtoFactory.java nifi/nar-bundles/framework-bundle/framework/web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerFacade.java commit 1392f328c905a7919d2555767cb53395409e3e1e Merge: 39a77e4 b29b61a Author: Matt Gilman Date: Tue Jan 20 08:34:05 2015 -0500 Merge branch 'NIFI-250' of https://git-wip-us.apache.org/repos/asf/incubator-nifi into NIFI-250 Conflicts: nifi/nar-bundles/framework-bundle/framework/web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/DtoFactory.java nifi/nar-bundles/framework-bundle/framework/web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerFacade.java commit 39a77e4828e41f635f2f85c68b78ca9e21ad1970 Author: Matt Gilman Date: Tue Jan 13 14:52:23 2015 -0500 NIFI-250: - Creating endpoints for returning the available controller services and reporting tasks. - Made the Setting tabbed to provide a place for configuring controller services and reporting tasks. commit 58b02e41eec54b1eff86a90205ab69f007ad1b00 Author: Matt Gilman Date: Fri Jan 16 12:02:33 2015 -0500 NIFI-250: merging from develop commit b204688bfa738c47bed71a1aab6c46171b21107f Author: Matt Gilman Date: Fri Jan 16 11:59:51 2015 -0500 NIFI-250: - Starting to making the available controller services collapsible. Still need to clean up and handle selection properly. commit b39fdbd6052adf342f72f44764145a3564669920 Author: Matt Gilman Date: Tue Jan 13 14:52:23 2015 -0500 NIFI-250: - Creating endpoints for returning the available controller services and reporting tasks. - Made the Setting tabbed to provide a place for configuring controller services and reporting tasks. NIFI-250: - Extracting the tagcloud into a re-usable jQuery plugin. NIFI-250: - Creating a new controller service dialog. Dialog may be refactored into a reusable widget if possible. - Loading the new controller service dialog with all available controller services. - Fixing typos. commit b29b61a5a8290b1b0a5513786227d7036e37aa6b Merge: 8976ff4 bb108a0 Author: Matt Gilman Date: Mon Jan 19 13:15:03 2015 -0500 Merge branch 'NIFI-250' of https://git-wip-us.apache.org/repos/asf/incubator-nifi into NIFI-250 Conflicts: nar-bundles/framework-bundle/framework/web/nifi-web-ui/src/main/webapp/WEB-INF/partials/canvas/settings-content.jsp nar-bundles/framework-bundle/framework/web/nifi-web-ui/src/main/webapp/css/settings.css nar-bundles/framework-bundle/framework/web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-settings.js nifi/nar-bundles/framework-bundle/framework/web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/DtoFactory.java nifi/nar-bundles/framework-bundle/framework/web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerFacade.java commit 8976ff49834ad4116b0fe9425be00e86da74b1a7 Author: Matt Gilman Date: Fri Jan 16 12:02:33 2015 -0500 NIFI-250: merging from develop commit 367baad5b6d9d633610e3547c75776338989f785 Author: Matt Gilman Date: Fri Jan 16 11:59:51 2015 -0500 NIFI-250: - Starting to making the available controller services collapsible. Still need to clean up and handle selection properly. commit 14da3f3de97f4b72afe591026c2690f92f4f7606 Author: Matt Gilman Date: Tue Jan 13 14:52:23 2015 -0500 NIFI-250: - Creating endpoints for returning the available controller services and reporting tasks. - Made the Setting tabbed to provide a place for configuring controller services and reporting tasks. NIFI-250: - Extracting the tagcloud into a re-usable jQuery plugin. NIFI-250: - Creating a new controller service dialog. Dialog may be refactored into a reusable widget if possible. - Loading the new controller service dialog with all available controller services. - Fixing typos. commit bb108a0960d6095791afd367bee91e62d8d605da Merge: da18ce0 03d422e Author: Matt Gilman Date: Wed Jan 14 13:34:45 2015 -0500 Merge branch 'NIFI-250' of https://git-wip-us.apache.org/repos/asf/incubator-nifi into NIFI-250 commit da18ce0ab0fee2e497e65ff4af30dd8dac212e96 Author: Matt Gilman Date: Tue Jan 13 14:52:23 2015 -0500 NIFI-250: - Creating endpoints for returning the available controller services and reporting tasks. - Made the Setting tabbed to provide a place for configuring controller services and reporting tasks. commit 03d422e4d4d65ebe4b080fc5ec88cc7aae30ae82 Author: Matt Gilman Date: Tue Jan 13 14:52:23 2015 -0500 NIFI-250: - Creating endpoints for returning the available controller services and reporting tasks. - Made the Setting tabbed to provide a place for configuring controller services and reporting tasks. --- .../nifi/annotation/lifecycle/OnAdded.java | 13 +- .../nifi/annotation/lifecycle/OnDisabled.java | 29 +- .../nifi/annotation/lifecycle/OnEnabled.java | 38 +- .../nifi/annotation/lifecycle/OnRemoved.java | 14 +- .../nifi/annotation/lifecycle/OnShutdown.java | 13 +- .../nifi/annotation/lifecycle/OnStopped.java | 9 + .../annotation/lifecycle/OnUnscheduled.java | 2 - .../nifi/components/PropertyDescriptor.java | 14 +- .../nifi/components/ValidationContext.java | 9 + .../controller/AbstractControllerService.java | 13 +- ...ontrollerServiceInitializationContext.java | 10 + .../controller/ControllerServiceLookup.java | 19 + .../org/apache/nifi/logging/ComponentLog.java | 100 + .../org/apache/nifi/logging/ProcessorLog.java | 61 +- .../nifi/reporting/AbstractReportingTask.java | 10 + .../ReportingInitializationContext.java | 10 + .../nifi/web/ClusterRequestException.java | 1 + .../org/apache/nifi/web/ComponentDetails.java | 157 ++ .../apache/nifi/web/ConfigurationAction.java | 137 ++ .../nifi/web/NiFiWebConfigurationContext.java | 102 + .../NiFiWebConfigurationRequestContext.java | 31 + .../org/apache/nifi/web/NiFiWebContext.java | 1 + .../apache/nifi/web/NiFiWebContextConfig.java | 1 + .../nifi/web/NiFiWebRequestContext.java | 58 + .../web/ProcessorConfigurationAction.java | 1 + .../org/apache/nifi/web/ProcessorInfo.java | 1 + .../java/org/apache/nifi/web/Revision.java | 32 +- .../org/apache/nifi/web/UiExtensionType.java | 31 + nifi/nifi-assembly/pom.xml | 2 - .../processor/util/StandardValidators.java | 20 +- .../org/apache/nifi/util/NiFiProperties.java | 2 - .../client/socket/EndpointConnectionPool.java | 2 +- .../TestEndpointConnectionStatePool.java | 4 +- ...ontrollerServiceInitializationContext.java | 17 + .../util/MockControllerServiceLookup.java | 11 + .../MockProcessorInitializationContext.java | 10 + .../apache/nifi/util/MockProcessorLog.java | 34 +- .../MockReportingInitializationContext.java | 10 +- .../nifi/util/MockValidationContext.java | 15 + .../util/StandardProcessorTestRunner.java | 5 +- .../org/apache/nifi/admin/dao/ActionDAO.java | 6 +- .../admin/dao/impl/StandardActionDAO.java | 60 +- .../nifi/admin/service/AuditService.java | 6 +- .../service/action/GetPreviousValues.java | 8 +- .../service/impl/StandardAuditService.java | 4 +- ...storyDTO.java => ComponentHistoryDTO.java} | 20 +- .../api/dto/ControllerConfigurationDTO.java | 18 + .../web/api/dto/ControllerServiceDTO.java | 190 ++ ...trollerServiceReferencingComponentDTO.java | 207 ++ .../nifi/web/api/dto/DocumentedTypeDTO.java | 20 +- .../nifi/web/api/dto/FlowSnippetDTO.java | 15 +- .../nifi/web/api/dto/NiFiComponentDTO.java | 4 +- .../nifi/web/api/dto/ProcessorConfigDTO.java | 218 +-- .../web/api/dto/PropertyDescriptorDTO.java | 243 +++ .../nifi/web/api/dto/ReportingTaskDTO.java | 228 +++ .../apache/nifi/web/api/dto/RevisionDTO.java | 15 + .../details/ComponentDetailsDTO.java | 2 +- ...tailsDTO.java => ExtensionDetailsDTO.java} | 8 +- ...ntity.java => ComponentHistoryEntity.java} | 22 +- .../api/entity/ControllerServiceEntity.java | 45 + ...lerServiceReferencingComponentsEntity.java | 46 + .../entity/ControllerServiceTypesEntity.java | 46 + .../api/entity/ControllerServicesEntity.java | 46 + .../api/entity/PropertyDescriptorEntity.java | 46 + .../web/api/entity/ReportingTaskEntity.java | 45 + .../api/entity/ReportingTaskTypesEntity.java | 46 + .../web/api/entity/ReportingTasksEntity.java | 46 + ...ontrollerServiceInitializationContext.java | 6 + .../mock/MockControllerServiceLookup.java | 10 + .../MockReportingInitializationContext.java | 8 +- .../nifi-framework-cluster-web/pom.xml | 4 - .../context/ClusterContextThreadLocal.java | 7 +- .../ClusterAwareOptimisticLockingManager.java | 96 - .../nifi-framework-cluster/pom.xml | 4 + .../nifi/cluster/flow/ClusterDataFlow.java | 15 +- .../flow/DataFlowManagementService.java | 17 + .../cluster/flow/impl/DataFlowDaoImpl.java | 43 +- .../impl/DataFlowManagementServiceImpl.java | 65 +- .../nifi/cluster/manager/NodeResponse.java | 14 + .../manager/impl/WebClusterManager.java | 935 +++++++-- .../spring/WebClusterManagerFactoryBean.java | 23 +- .../nifi-cluster-manager-context.xml | 4 + .../nifi-framework-core-api/.gitignore | 1 + .../AbstractConfiguredComponent.java | 18 +- .../nifi/controller/ProcessScheduler.java | 27 +- .../apache/nifi/controller/ProcessorNode.java | 16 + .../nifi/controller/ReportingTaskNode.java | 26 +- .../controller/ValidationContextFactory.java | 4 + ...trollerServiceInstantiationException.java} | 10 +- .../reporting/ReportingTaskProvider.java | 103 + .../service/ControllerServiceNode.java | 39 +- .../service/ControllerServiceProvider.java | 83 +- .../service/ControllerServiceReference.java | 7 +- .../service/ControllerServiceState.java | 45 + .../nifi/controller/FlowController.java | 205 +- .../nifi/controller/FlowFromDOMFactory.java | 60 +- .../controller/StandardFlowSerializer.java | 68 +- .../nifi/controller/StandardFlowService.java | 9 +- .../controller/StandardFlowSynchronizer.java | 236 ++- .../controller/StandardProcessorNode.java | 47 +- .../nifi/controller/TemplateManager.java | 33 +- .../reporting/AbstractReportingTaskNode.java | 72 +- .../reporting/StandardReportingContext.java | 11 + ...tandardReportingInitializationContext.java | 23 +- .../scheduling/StandardProcessScheduler.java | 208 +- .../service/ControllerServiceLoader.java | 147 +- ...ontrollerServiceInitializationContext.java | 20 +- .../StandardControllerServiceNode.java | 122 +- .../StandardControllerServiceProvider.java | 452 ++++- .../StandardControllerServiceReference.java | 19 +- .../tasks/ReportingTaskWrapper.java | 29 +- .../nifi/fingerprint/FingerprintFactory.java | 79 + .../nifi/groups/StandardProcessGroup.java | 28 +- .../persistence/FlowConfigurationDAO.java | 25 - .../StandardXMLFlowConfigurationDAO.java | 191 +- .../nifi/processor/SimpleProcessLogger.java | 36 +- .../processor/StandardProcessContext.java | 10 + .../processor/StandardSchedulingContext.java | 5 +- .../processor/StandardValidationContext.java | 12 + .../StandardValidationContextFactory.java | 5 + .../java/org/apache/nifi/util/DomUtils.java | 10 + .../ControllerServiceConfiguration.xsd | 61 - .../src/main/resources/FlowConfiguration.xsd | 49 +- .../resources/ReportingTaskConfiguration.xsd | 87 - ...StandardControllerServiceProviderTest.java | 2 +- ...TestStandardControllerServiceProvider.java | 385 ++++ .../service/mock/DummyProcessor.java | 49 + .../controller/service/mock/ServiceA.java | 49 + .../controller/service/mock/ServiceB.java} | 47 +- .../processor/TestStandardPropertyValue.java | 11 +- .../nifi-framework/nifi-nar-utils/.gitignore | 1 + .../resources/conf/controller-services.xml | 18 - .../src/main/resources/conf/nifi.properties | 2 - .../main/resources/conf/reporting-tasks.xml | 49 - .../org/apache/nifi/action/Component.java | 4 +- ...ssorDetails.java => ExtensionDetails.java} | 2 +- ...ttpServletConfigurationRequestContext.java | 56 + .../nifi/web/HttpServletRequestContext.java | 100 + .../web/HttpServletRequestContextConfig.java | 1 + .../nifi-web/nifi-jetty/pom.xml | 5 + .../apache/nifi/web/server/JettyServer.java | 208 +- .../nifi-web/nifi-ui-extension/pom.xml | 21 + .../apache/nifi/ui/extension/UiExtension.java | 52 + .../nifi/ui/extension/UiExtensionMapping.java | 52 + .../nifi-web/nifi-web-api/pom.xml | 5 + .../nifi/audit/ControllerServiceAuditor.java | 475 +++++ .../org/apache/nifi/audit/FunnelAuditor.java | 8 +- .../org/apache/nifi/audit/NiFiAuditor.java | 11 +- .../org/apache/nifi/audit/PortAuditor.java | 17 +- .../nifi/audit/ProcessGroupAuditor.java | 18 +- .../apache/nifi/audit/ProcessorAuditor.java | 20 +- .../nifi/audit/RelationshipAuditor.java | 18 +- .../nifi/audit/RemoteProcessGroupAuditor.java | 22 +- .../nifi/audit/ReportingTaskAuditor.java | 353 ++++ .../org/apache/nifi/audit/SnippetAuditor.java | 13 +- .../apache/nifi/web/NiFiServiceFacade.java | 206 +- .../nifi/web/StandardNiFiServiceFacade.java | 1381 +++++++------ .../StandardNiFiWebConfigurationContext.java | 736 +++++++ .../nifi/web/StandardNiFiWebContext.java | 28 +- .../nifi/web/api/ApplicationResource.java | 86 +- .../apache/nifi/web/api/ClusterResource.java | 2 +- .../nifi/web/api/ConnectionResource.java | 6 +- .../nifi/web/api/ControllerResource.java | 99 +- .../web/api/ControllerServiceResource.java | 803 ++++++++ .../apache/nifi/web/api/FunnelResource.java | 6 +- .../apache/nifi/web/api/HistoryResource.java | 70 +- .../nifi/web/api/InputPortResource.java | 6 +- .../apache/nifi/web/api/LabelResource.java | 6 +- .../nifi/web/api/OutputPortResource.java | 6 +- .../nifi/web/api/ProcessGroupResource.java | 16 +- .../nifi/web/api/ProcessorResource.java | 72 +- .../web/api/RemoteProcessGroupResource.java | 10 +- .../nifi/web/api/ReportingTaskResource.java | 663 +++++++ .../apache/nifi/web/api/SnippetResource.java | 6 +- .../nifi/web/api/config/ThrowableMapper.java | 7 +- .../apache/nifi/web/api/dto/DtoFactory.java | 426 +++- .../nifi/web/controller/ControllerFacade.java | 75 +- .../nifi/web/dao/ControllerServiceDAO.java | 110 ++ .../apache/nifi/web/dao/ReportingTaskDAO.java | 88 + .../impl/StandardControllerServiceDAO.java | 320 +++ .../web/dao/impl/StandardProcessorDAO.java | 5 + .../dao/impl/StandardReportingTaskDAO.java | 365 ++++ .../nifi/web/dao/impl/StandardSnippetDAO.java | 36 +- .../ControllerServiceProviderFactoryBean.java | 68 + .../OptimisticLockingManagerFactoryBean.java | 67 + .../ReportingTaskProviderFactoryBean.java | 69 + .../apache/nifi/web/util/Availability.java | 34 + .../apache/nifi/web/util/SnippetUtils.java | 181 +- .../main/resources/nifi-web-api-context.xml | 60 +- .../nifi/integration/util/NiFiTestServer.java | 5 + .../nifi-web-optimistic-locking/pom.xml | 12 + .../apache/nifi/web/ConfigurationRequest.java | 34 + .../nifi/web/ConfigurationSnapshot.java | 22 +- .../org/apache/nifi/web/FlowModification.java | 57 + .../nifi/web/OptimisticLockingManager.java | 80 +- .../web/StandardOptimisticLockingManager.java | 158 +- .../org/apache/nifi/web/UpdateRevision.java | 31 + .../nifi/web/security/user/NiFiUserUtils.java | 10 + .../nifi-web/nifi-web-ui/pom.xml | 12 +- .../main/resources/filters/canvas.properties | 8 +- .../src/main/webapp/WEB-INF/pages/canvas.jsp | 11 +- .../src/main/webapp/WEB-INF/pages/summary.jsp | 2 + .../WEB-INF/partials/canvas/canvas-header.jsp | 2 +- .../controller-service-configuration.jsp | 90 + .../disable-controller-service-dialog.jsp | 71 + .../enable-controller-service-dialog.jsp | 70 + .../canvas/new-controller-service-dialog.jsp | 53 + .../partials/canvas/new-processor-dialog.jsp | 6 +- .../canvas/new-processor-property-dialog.jsp | 34 - .../canvas/new-reporting-task-dialog.jsp | 53 + .../canvas/processor-configuration.jsp | 21 +- .../WEB-INF/partials/canvas/registration.jsp | 2 +- .../canvas/reporting-task-configuration.jsp | 107 + .../partials/canvas/settings-content.jsp | 104 +- .../WEB-INF/partials/processor-details.jsp | 4 - .../nifi-web-ui/src/main/webapp/css/about.css | 1 - .../src/main/webapp/css/canvas.css | 4 + .../webapp/css/connection-configuration.css | 1 - .../main/webapp/css/connection-details.css | 1 - .../main/webapp/css/controller-service.css | 269 +++ .../src/main/webapp/css/dialog.css | 25 +- .../main/webapp/css/label-configuration.css | 1 - .../nifi-web-ui/src/main/webapp/css/main.css | 42 +- .../css/new-controller-service-dialog.css | 152 ++ .../main/webapp/css/new-processor-dialog.css | 53 +- .../webapp/css/new-reporting-task-dialog.css | 152 ++ .../main/webapp/css/port-configuration.css | 2 - .../src/main/webapp/css/port-details.css | 2 - .../css/process-group-configuration.css | 1 - .../main/webapp/css/process-group-details.css | 1 - .../webapp/css/processor-configuration.css | 153 -- .../src/main/webapp/css/processor-details.css | 27 - .../src/main/webapp/css/registration.css | 8 - .../remote-process-group-configuration.css | 3 - .../src/main/webapp/css/reporting-task.css | 116 ++ .../src/main/webapp/css/settings.css | 138 +- .../nifi-web-ui/src/main/webapp/css/shell.css | 2 +- .../src/main/webapp/css/status-history.css | 1 - .../src/main/webapp/images/iconEnable.png | Bin 0 -> 472 bytes .../src/main/webapp/images/iconUndo.png | Bin 642 -> 0 bytes .../src/main/webapp/js/jquery/jquery.each.js | 2 +- .../webapp/js/jquery/modal/jquery.modal.css | 12 +- .../webapp/js/jquery/modal/jquery.modal.js | 121 +- .../js/jquery/nfeditor/jquery.nfeditor.js | 5 +- .../propertytable}/buttonNewProperty.png | Bin .../propertytable/jquery.propertytable.css | 173 ++ .../propertytable/jquery.propertytable.js | 1317 +++++++++++++ .../webapp/js/jquery/tabbs/jquery.tabbs.js | 2 + .../js/jquery/tagcloud/jquery.tagcloud.css | 62 + .../js/jquery/tagcloud/jquery.tagcloud.js | 226 +++ .../js/nf/bulletin-board/nf-bulletin-board.js | 3 + .../main/webapp/js/nf/canvas/nf-actions.js | 3 + .../main/webapp/js/nf/canvas/nf-birdseye.js | 3 + .../webapp/js/nf/canvas/nf-canvas-header.js | 64 +- .../webapp/js/nf/canvas/nf-canvas-toolbar.js | 3 + .../webapp/js/nf/canvas/nf-canvas-toolbox.js | 164 +- .../webapp/js/nf/canvas/nf-canvas-utils.js | 3 + .../src/main/webapp/js/nf/canvas/nf-canvas.js | 43 +- .../main/webapp/js/nf/canvas/nf-clipboard.js | 3 + .../webapp/js/nf/canvas/nf-connectable.js | 3 + .../nf/canvas/nf-connection-configuration.js | 3 + .../main/webapp/js/nf/canvas/nf-connection.js | 3 + .../webapp/js/nf/canvas/nf-context-menu.js | 3 + .../js/nf/canvas/nf-controller-service.js | 1730 +++++++++++++++++ ...custom-processor-ui.js => nf-custom-ui.js} | 12 +- .../main/webapp/js/nf/canvas/nf-draggable.js | 3 + .../src/main/webapp/js/nf/canvas/nf-funnel.js | 3 + .../src/main/webapp/js/nf/canvas/nf-go-to.js | 3 + .../webapp/js/nf/canvas/nf-graph-control.js | 3 + .../src/main/webapp/js/nf/canvas/nf-graph.js | 3 + .../js/nf/canvas/nf-label-configuration.js | 3 + .../src/main/webapp/js/nf/canvas/nf-label.js | 3 + .../js/nf/canvas/nf-port-configuration.js | 3 + .../webapp/js/nf/canvas/nf-port-details.js | 3 + .../src/main/webapp/js/nf/canvas/nf-port.js | 3 + .../canvas/nf-process-group-configuration.js | 3 + .../js/nf/canvas/nf-process-group-details.js | 3 + .../webapp/js/nf/canvas/nf-process-group.js | 3 + .../nf/canvas/nf-processor-configuration.js | 505 ++--- .../nf-processor-property-combo-editor.js | 177 -- .../nf-processor-property-nfel-editor.js | 207 -- .../nf/canvas/nf-processor-property-table.js | 567 ------ .../nf-processor-property-text-editor.js | 212 -- .../main/webapp/js/nf/canvas/nf-processor.js | 3 + .../webapp/js/nf/canvas/nf-registration.js | 7 +- .../nf-remote-process-group-configuration.js | 3 + .../canvas/nf-remote-process-group-details.js | 3 + .../canvas/nf-remote-process-group-ports.js | 3 + .../js/nf/canvas/nf-remote-process-group.js | 3 + .../webapp/js/nf/canvas/nf-reporting-task.js | 700 +++++++ .../js/nf/canvas/nf-secure-port-details.js | 3 + .../main/webapp/js/nf/canvas/nf-selectable.js | 3 + .../main/webapp/js/nf/canvas/nf-settings.js | 1707 +++++++++++++++- .../main/webapp/js/nf/canvas/nf-snippet.js | 3 + .../main/webapp/js/nf/canvas/nf-storage.js | 3 + .../webapp/js/nf/canvas/nf-toolbar-action.js | 3 + .../webapp/js/nf/cluster/nf-cluster-table.js | 3 + .../main/webapp/js/nf/cluster/nf-cluster.js | 3 + .../js/nf/counters/nf-counters-table.js | 3 + .../main/webapp/js/nf/counters/nf-counters.js | 3 + .../webapp/js/nf/history/nf-history-model.js | 3 + .../webapp/js/nf/history/nf-history-table.js | 7 +- .../main/webapp/js/nf/history/nf-history.js | 3 + .../src/main/webapp/js/nf/nf-client.js | 3 + .../src/main/webapp/js/nf/nf-common.js | 74 +- .../webapp/js/nf/nf-connection-details.js | 3 + .../src/main/webapp/js/nf/nf-dialog.js | 18 +- .../main/webapp/js/nf/nf-processor-details.js | 357 +--- .../src/main/webapp/js/nf/nf-shell.js | 3 + .../main/webapp/js/nf/nf-status-history.js | 3 + .../js/nf/provenance/nf-provenance-lineage.js | 3 + .../js/nf/provenance/nf-provenance-table.js | 3 + .../webapp/js/nf/provenance/nf-provenance.js | 3 + .../webapp/js/nf/summary/nf-summary-table.js | 2 + .../main/webapp/js/nf/summary/nf-summary.js | 3 + .../js/nf/templates/nf-templates-table.js | 3 + .../webapp/js/nf/templates/nf-templates.js | 3 + .../main/webapp/js/nf/users/nf-users-table.js | 3 + .../nifi-framework/nifi-web/pom.xml | 4 +- .../standard/TestDetectDuplicate.java | 5 +- .../DistributedMapCacheClientService.java | 21 +- .../DistributedSetCacheClientService.java | 16 +- .../cache/server/AbstractCacheServer.java | 2 +- .../cache/server/DistributedCacheServer.java | 13 +- .../nifi/http/StandardHttpContextMap.java | 9 + .../nifi/ssl/StandardSSLContextService.java | 39 +- .../update/attributes/api/RuleResource.java | 131 +- ...processor => nifi-processor-configuration} | 0 .../src/main/webapp/WEB-INF/jsp/worksheet.jsp | 3 +- .../src/main/webapp/WEB-INF/web.xml | 4 +- .../src/main/webapp/js/application.js | 12 +- nifi/pom.xml | 5 + 332 files changed, 20850 insertions(+), 5142 deletions(-) create mode 100644 nifi/nifi-api/src/main/java/org/apache/nifi/logging/ComponentLog.java create mode 100644 nifi/nifi-api/src/main/java/org/apache/nifi/web/ComponentDetails.java create mode 100644 nifi/nifi-api/src/main/java/org/apache/nifi/web/ConfigurationAction.java create mode 100644 nifi/nifi-api/src/main/java/org/apache/nifi/web/NiFiWebConfigurationContext.java create mode 100644 nifi/nifi-api/src/main/java/org/apache/nifi/web/NiFiWebConfigurationRequestContext.java create mode 100644 nifi/nifi-api/src/main/java/org/apache/nifi/web/NiFiWebRequestContext.java create mode 100644 nifi/nifi-api/src/main/java/org/apache/nifi/web/UiExtensionType.java rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/{ProcessorHistoryDTO.java => ComponentHistoryDTO.java} (77%) create mode 100644 nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ControllerServiceDTO.java create mode 100644 nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ControllerServiceReferencingComponentDTO.java create mode 100644 nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/PropertyDescriptorDTO.java create mode 100644 nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ReportingTaskDTO.java rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/action/component/details/{ProcessorDetailsDTO.java => ExtensionDetailsDTO.java} (87%) rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/{ProcessorHistoryEntity.java => ComponentHistoryEntity.java} (66%) create mode 100644 nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ControllerServiceEntity.java create mode 100644 nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ControllerServiceReferencingComponentsEntity.java create mode 100644 nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ControllerServiceTypesEntity.java create mode 100644 nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ControllerServicesEntity.java create mode 100644 nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/PropertyDescriptorEntity.java create mode 100644 nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ReportingTaskEntity.java create mode 100644 nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ReportingTaskTypesEntity.java create mode 100644 nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ReportingTasksEntity.java delete mode 100644 nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-web/src/main/java/org/apache/nifi/web/ClusterAwareOptimisticLockingManager.java rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/exception/{ControllerServiceNotFoundException.java => ControllerServiceInstantiationException.java} (77%) create mode 100644 nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/reporting/ReportingTaskProvider.java create mode 100644 nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/service/ControllerServiceState.java delete mode 100644 nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/resources/ControllerServiceConfiguration.xsd delete mode 100644 nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/resources/ReportingTaskConfiguration.xsd create mode 100644 nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/TestStandardControllerServiceProvider.java create mode 100644 nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/mock/DummyProcessor.java create mode 100644 nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/mock/ServiceA.java rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-framework-core-api/src/main/java/org/apache/nifi/controller/Availability.java => nifi-framework-core/src/test/java/org/apache/nifi/controller/service/mock/ServiceB.java} (80%) delete mode 100644 nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/controller-services.xml delete mode 100644 nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/reporting-tasks.xml rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-user-actions/src/main/java/org/apache/nifi/action/component/details/{ProcessorDetails.java => ExtensionDetails.java} (94%) create mode 100644 nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-custom-ui-utilities/src/main/java/org/apache/nifi/web/HttpServletConfigurationRequestContext.java create mode 100644 nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-custom-ui-utilities/src/main/java/org/apache/nifi/web/HttpServletRequestContext.java create mode 100644 nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-ui-extension/pom.xml create mode 100644 nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-ui-extension/src/main/java/org/apache/nifi/ui/extension/UiExtension.java create mode 100644 nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-ui-extension/src/main/java/org/apache/nifi/ui/extension/UiExtensionMapping.java create mode 100644 nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/ControllerServiceAuditor.java create mode 100644 nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/ReportingTaskAuditor.java create mode 100644 nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiWebConfigurationContext.java create mode 100644 nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ControllerServiceResource.java create mode 100644 nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ReportingTaskResource.java create mode 100644 nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/ControllerServiceDAO.java create mode 100644 nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/ReportingTaskDAO.java create mode 100644 nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardControllerServiceDAO.java create mode 100644 nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardReportingTaskDAO.java create mode 100644 nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/spring/ControllerServiceProviderFactoryBean.java create mode 100644 nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/spring/OptimisticLockingManagerFactoryBean.java create mode 100644 nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/spring/ReportingTaskProviderFactoryBean.java create mode 100644 nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/util/Availability.java create mode 100644 nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-optimistic-locking/src/main/java/org/apache/nifi/web/ConfigurationRequest.java create mode 100644 nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-optimistic-locking/src/main/java/org/apache/nifi/web/FlowModification.java create mode 100644 nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-optimistic-locking/src/main/java/org/apache/nifi/web/UpdateRevision.java create mode 100644 nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/partials/canvas/controller-service-configuration.jsp create mode 100644 nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/partials/canvas/disable-controller-service-dialog.jsp create mode 100644 nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/partials/canvas/enable-controller-service-dialog.jsp create mode 100644 nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/partials/canvas/new-controller-service-dialog.jsp delete mode 100644 nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/partials/canvas/new-processor-property-dialog.jsp create mode 100644 nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/partials/canvas/new-reporting-task-dialog.jsp create mode 100644 nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/partials/canvas/reporting-task-configuration.jsp create mode 100644 nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/css/controller-service.css create mode 100644 nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/css/new-controller-service-dialog.css create mode 100644 nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/css/new-reporting-task-dialog.css create mode 100644 nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/css/reporting-task.css create mode 100644 nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/images/iconEnable.png delete mode 100755 nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/images/iconUndo.png rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/{images => js/jquery/propertytable}/buttonNewProperty.png (100%) create mode 100644 nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/jquery/propertytable/jquery.propertytable.css create mode 100644 nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/jquery/propertytable/jquery.propertytable.js create mode 100644 nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/jquery/tagcloud/jquery.tagcloud.css create mode 100644 nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/jquery/tagcloud/jquery.tagcloud.js create mode 100644 nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-controller-service.js rename nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/{nf-custom-processor-ui.js => nf-custom-ui.js} (86%) delete mode 100644 nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-processor-property-combo-editor.js delete mode 100644 nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-processor-property-nfel-editor.js delete mode 100644 nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-processor-property-table.js delete mode 100644 nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-processor-property-text-editor.js create mode 100644 nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-reporting-task.js rename nifi/nifi-nar-bundles/nifi-update-attribute-bundle/nifi-update-attribute-ui/src/main/webapp/META-INF/{nifi-processor => nifi-processor-configuration} (100%) diff --git a/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/lifecycle/OnAdded.java b/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/lifecycle/OnAdded.java index acb7a4d027..a1286ea473 100644 --- a/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/lifecycle/OnAdded.java +++ b/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/lifecycle/OnAdded.java @@ -24,16 +24,25 @@ import java.lang.annotation.RetentionPolicy; import java.lang.annotation.Target; /** + *

      * Marker annotation a {@link org.apache.nifi.processor.Processor Processor}, * {@link org.apache.nifi.controller.ControllerService ControllerService}, or * {@link org.apache.nifi.reporting.ReportingTask ReportingTask} * implementation can use to indicate a method * should be called whenever the component is added to the flow. This method * will be called once for the entire life of a component instance. - * + *

      + * + *

      + * Methods with this annotation are called without any arguments, as all settings + * and properties can be assumed to be the defaults. + *

      + * + *

      * If any method annotated with this annotation throws a Throwable, the component * will not be added to the flow. - * + *

      + * * @author none */ @Documented diff --git a/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/lifecycle/OnDisabled.java b/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/lifecycle/OnDisabled.java index 0f78010026..b227968931 100644 --- a/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/lifecycle/OnDisabled.java +++ b/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/lifecycle/OnDisabled.java @@ -23,19 +23,32 @@ import java.lang.annotation.Retention; import java.lang.annotation.RetentionPolicy; import java.lang.annotation.Target; +import org.apache.nifi.controller.ConfigurationContext; + /** - * Marker annotation a {@link org.apache.nifi.processor.Processor Processor}, - * {@link org.apache.nifi.controller.ControllerService ControllerService} or - * {@link org.apache.nifi.reporting.ReportingTask ReportingTask} - * can use to indicate a method should be called whenever the component is disabled. + *

      + * Marker annotation a {@link org.apache.nifi.controller.ControllerService ControllerService} + * can use to indicate a method should be called whenever the service is disabled. + *

      * *

      - * Methods using this annotation must take no arguments. If a method with this annotation - * throws a Throwable, a log message and bulletin will be issued for the component, but - * the component will still be disabled. + * Methods using this annotation are permitted to take zero arguments or to take a single + * argument of type {@link ConfigurationContext}. If a method with this annotation + * throws a Throwable, a log message and bulletin will be issued for the service, and the + * service will remain in a 'DISABLING' state. When this occurs, the method with this annotation + * will be called again after some period of time. This will continue until the method returns + * without throwing any Throwable. Until that time, the service will remain in a 'DISABLING' state + * and cannot be enabled again. + *

      + * + *

      + * Note that this annotation will be ignored if applied to a ReportingTask or Processor. For a Controller + * Service, enabling and disabling are considered lifecycle events, as the action makes them usable or + * unusable by other components. However, for a Processor and a Reporting + * Task, these are not lifecycle events but rather a mechanism to allow a component to be excluded when + * starting or stopping a group of components. *

      * - * @author none */ @Documented @Target({ElementType.METHOD}) diff --git a/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/lifecycle/OnEnabled.java b/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/lifecycle/OnEnabled.java index 1536decb80..32aeec6e7c 100644 --- a/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/lifecycle/OnEnabled.java +++ b/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/lifecycle/OnEnabled.java @@ -25,35 +25,35 @@ import java.lang.annotation.Target; /** *

      - * Marker annotation a {@link org.apache.nifi.processor.Processor Processor}, - * {@link org.apache.nifi.controller.ControllerService ControllerService} or - * {@link org.apache.nifi.reporting.ReportingTask ReportingTask} - * can use to indicate a method should be called whenever the component is enabled. - * Any method that has this annotation will be called every time a user enables the component. + * Marker annotation a {@link org.apache.nifi.controller.ControllerService ControllerService} + * can use to indicate a method should be called whenever the service is enabled. + * Any method that has this annotation will be called every time a user enables the service. * Additionally, each time that NiFi is restarted, if NiFi is configured to "auto-resume state" - * and the component is enabled (whether stopped or running), the method will be invoked. + * and the service is enabled, the method will be invoked. *

      * *

      - * Methods using this annotation must take either 0 arguments or a single argument. - *

      - * - *

      - * If using 1 argument and the component using the annotation is a Processor, that argument must - * be of type {@link org.apache.nifi.processor.ProcessContext ProcessContext}. - *

      - * - *

      - * If using 1 argument and the component using the annotation is a Reporting Task or Controller Service, - * that argument must be of type {@link org.apache.nifi.controller.ConfigurationContext ConfigurationContext}. + * Methods using this annotation must take either 0 arguments or a single argument of type + * {@link org.apache.nifi.controller.ConfigurationContext ConfigurationContext}. *

      * *

      * If a method with this annotation throws a Throwable, a log message and bulletin will be issued - * for the component, but the component will still be enabled. + * for the component. In this event, the service will remain in an 'ENABLING' state and will not be + * usable. All methods with this annotation will then be called again after a delay. The service will + * not be made available for use until all methods with this annotation have returned without throwing + * anything. + *

      + * + *

      + * Note that this annotation will be ignored if applied to a ReportingTask or Processor. For a Controller + * Service, enabling and disabling are considered lifecycle events, as the action makes them usable or + * unusable by other components. However, for a Processor and a Reporting + * Task, these are not lifecycle events but rather a mechanism to allow a component to be excluded when + * starting or stopping a group of components. *

      * - * @author none + * */ @Documented @Target({ElementType.METHOD}) diff --git a/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/lifecycle/OnRemoved.java b/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/lifecycle/OnRemoved.java index 696159f0e4..71202b4317 100644 --- a/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/lifecycle/OnRemoved.java +++ b/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/lifecycle/OnRemoved.java @@ -23,7 +23,11 @@ import java.lang.annotation.Retention; import java.lang.annotation.RetentionPolicy; import java.lang.annotation.Target; +import org.apache.nifi.controller.ConfigurationContext; +import org.apache.nifi.processor.ProcessContext; + /** + *

      * Marker annotation a {@link org.apache.nifi.processor.Processor Processor}, * {@link org.apache.nifi.controller.ControllerService ControllerService}, or * {@link org.apache.nifi.reporting.ReportingTask ReportingTask} implementation @@ -32,7 +36,15 @@ import java.lang.annotation.Target; * component instance. If the method throw any Throwable, that Throwable will be * caught and logged but will not prevent subsequent methods with this annotation * or removal of the component from the flow. - * + *

      + * + *

      + * Methods with this annotation are permitted to take no arguments or to take a single + * argument. If using a single argument, that argument must be of type {@link ConfigurationContext} + * if the component is a ReportingTask or a ControllerService. If the component is a Processor, + * then the argument must be of type {@link ProcessContext}. + *

      + * * @author none */ @Documented diff --git a/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/lifecycle/OnShutdown.java b/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/lifecycle/OnShutdown.java index a4129e158d..3d1ce6c642 100644 --- a/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/lifecycle/OnShutdown.java +++ b/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/lifecycle/OnShutdown.java @@ -23,7 +23,11 @@ import java.lang.annotation.Retention; import java.lang.annotation.RetentionPolicy; import java.lang.annotation.Target; +import org.apache.nifi.controller.ConfigurationContext; +import org.apache.nifi.processor.ProcessContext; + /** + *

      * Marker annotation a {@link org.apache.nifi.processor.Processor Processor}, * {@link org.apache.nifi.controller.ControllerService ControllerService}, or * {@link org.apache.nifi.reporting.ReportingTask ReportingTask} implementation @@ -31,7 +35,14 @@ import java.lang.annotation.Target; * This will be called at most once for each component in a JVM lifetime. * It is not, however, guaranteed that this method will be called on shutdown, as * the service may be killed suddenly. - * + *

      + * + *

      + * Methods with this annotation are permitted to take either 0 or 1 argument. If an argument + * is used, it must be of type {@link ConfigurationContext} if the component is a ReportingTask + * or Controller Service, or of type {@link ProcessContext} if the component is a Processor. + *

      + * * @author none */ @Documented diff --git a/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/lifecycle/OnStopped.java b/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/lifecycle/OnStopped.java index 4715253234..fdc4fd8e09 100644 --- a/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/lifecycle/OnStopped.java +++ b/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/lifecycle/OnStopped.java @@ -23,6 +23,9 @@ import java.lang.annotation.Retention; import java.lang.annotation.RetentionPolicy; import java.lang.annotation.Target; +import org.apache.nifi.controller.ConfigurationContext; +import org.apache.nifi.processor.ProcessContext; + /** *

      * Marker annotation a {@link org.apache.nifi.processor.Processor Processor} or @@ -47,6 +50,12 @@ import java.lang.annotation.Target; * longer scheduled to run (as opposed to after all threads have returned from the * onTrigger method), see the {@link OnUnscheduled} annotation. *

      + * + *

      + * Methods with this annotation are permitted to take either 0 or 1 argument. If an argument + * is used, it must be of type {@link ConfigurationContext} if the component is a ReportingTask + * or of type {@link ProcessContext} if the component is a Processor. + *

      * * @author none */ diff --git a/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/lifecycle/OnUnscheduled.java b/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/lifecycle/OnUnscheduled.java index b1dbde1bf2..5c7e13dc2c 100644 --- a/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/lifecycle/OnUnscheduled.java +++ b/nifi/nifi-api/src/main/java/org/apache/nifi/annotation/lifecycle/OnUnscheduled.java @@ -47,8 +47,6 @@ import java.lang.annotation.Target; * If using 1 argument and the component using the annotation is a Reporting Task, that argument must * be of type {@link org.apache.nifi.controller.ConfigurationContext ConfigurationContext}. *

      - * - * @author none */ @Documented @Target({ElementType.METHOD}) diff --git a/nifi/nifi-api/src/main/java/org/apache/nifi/components/PropertyDescriptor.java b/nifi/nifi-api/src/main/java/org/apache/nifi/components/PropertyDescriptor.java index 82372af540..e62ff7962c 100644 --- a/nifi/nifi-api/src/main/java/org/apache/nifi/components/PropertyDescriptor.java +++ b/nifi/nifi-api/src/main/java/org/apache/nifi/components/PropertyDescriptor.java @@ -142,9 +142,19 @@ public final class PropertyDescriptor implements Comparable final Set validIdentifiers = context.getControllerServiceLookup().getControllerServiceIdentifiers(controllerServiceDefinition); if (validIdentifiers != null && validIdentifiers.contains(input)) { final ControllerService controllerService = context.getControllerServiceLookup().getControllerService(input); - if (!context.getControllerServiceLookup().isControllerServiceEnabled(controllerService)) { + if ( !context.isValidationRequired(controllerService) ) { return new ValidationResult.Builder() - .input(input) + .input(input) + .subject(getName()) + .valid(true) + .build(); + } + + final String serviceId = controllerService.getIdentifier(); + if (!context.getControllerServiceLookup().isControllerServiceEnabled(serviceId) && + !context.getControllerServiceLookup().isControllerServiceEnabling(serviceId)) { + return new ValidationResult.Builder() + .input(context.getControllerServiceLookup().getControllerServiceName(serviceId)) .subject(getName()) .valid(false) .explanation("Controller Service " + controllerService + " is disabled") diff --git a/nifi/nifi-api/src/main/java/org/apache/nifi/components/ValidationContext.java b/nifi/nifi-api/src/main/java/org/apache/nifi/components/ValidationContext.java index b7b72c5141..61b68a249c 100644 --- a/nifi/nifi-api/src/main/java/org/apache/nifi/components/ValidationContext.java +++ b/nifi/nifi-api/src/main/java/org/apache/nifi/components/ValidationContext.java @@ -80,6 +80,15 @@ public interface ValidationContext { */ String getAnnotationData(); + /** + * There are times when the framework needs to consider a component valid, even if it + * references an invalid ControllerService. This method will return false + * if the component is to be considered valid even if the given Controller Service is referenced + * and is invalid. + * @param service + */ + boolean isValidationRequired(ControllerService service); + /** * Returns true if the given value contains a NiFi Expression Language expression, * false if it does not diff --git a/nifi/nifi-api/src/main/java/org/apache/nifi/controller/AbstractControllerService.java b/nifi/nifi-api/src/main/java/org/apache/nifi/controller/AbstractControllerService.java index c12f2f8d1f..71cdd231ea 100644 --- a/nifi/nifi-api/src/main/java/org/apache/nifi/controller/AbstractControllerService.java +++ b/nifi/nifi-api/src/main/java/org/apache/nifi/controller/AbstractControllerService.java @@ -22,6 +22,7 @@ import org.apache.nifi.components.AbstractConfigurableComponent; import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.components.PropertyValue; import org.apache.nifi.controller.annotation.OnConfigured; +import org.apache.nifi.logging.ComponentLog; import org.apache.nifi.processor.ProcessorInitializationContext; import org.apache.nifi.reporting.InitializationException; @@ -30,11 +31,13 @@ public abstract class AbstractControllerService extends AbstractConfigurableComp private String identifier; private ControllerServiceLookup serviceLookup; private volatile ConfigurationContext configContext; - + private ComponentLog logger; + @Override public final void initialize(final ControllerServiceInitializationContext context) throws InitializationException { this.identifier = context.getIdentifier(); serviceLookup = context.getControllerServiceLookup(); + logger = context.getLogger(); init(context); } @@ -88,4 +91,12 @@ public abstract class AbstractControllerService extends AbstractConfigurableComp */ protected void init(final ControllerServiceInitializationContext config) throws InitializationException { } + + /** + * Returns the logger that has been provided to the component by the framework in its initialize method. + * @return + */ + protected ComponentLog getLogger() { + return logger; + } } diff --git a/nifi/nifi-api/src/main/java/org/apache/nifi/controller/ControllerServiceInitializationContext.java b/nifi/nifi-api/src/main/java/org/apache/nifi/controller/ControllerServiceInitializationContext.java index b5b0412cb3..d34c6354ad 100644 --- a/nifi/nifi-api/src/main/java/org/apache/nifi/controller/ControllerServiceInitializationContext.java +++ b/nifi/nifi-api/src/main/java/org/apache/nifi/controller/ControllerServiceInitializationContext.java @@ -16,6 +16,8 @@ */ package org.apache.nifi.controller; +import org.apache.nifi.logging.ComponentLog; + public interface ControllerServiceInitializationContext { /** @@ -33,4 +35,12 @@ public interface ControllerServiceInitializationContext { * @return */ ControllerServiceLookup getControllerServiceLookup(); + + /** + * Returns a logger that can be used to log important events in a standard way and generate + * bulletins when appropriate + * + * @return + */ + ComponentLog getLogger(); } diff --git a/nifi/nifi-api/src/main/java/org/apache/nifi/controller/ControllerServiceLookup.java b/nifi/nifi-api/src/main/java/org/apache/nifi/controller/ControllerServiceLookup.java index 77b8e6296d..4b96f626a6 100644 --- a/nifi/nifi-api/src/main/java/org/apache/nifi/controller/ControllerServiceLookup.java +++ b/nifi/nifi-api/src/main/java/org/apache/nifi/controller/ControllerServiceLookup.java @@ -41,6 +41,18 @@ public interface ControllerServiceLookup { */ boolean isControllerServiceEnabled(String serviceIdentifier); + /** + * Returns true if the Controller Service with the given + * identifier has been enabled but is still in the transitioning state, + * otherwise returns false. + * If the given identifier is not known by this ControllerServiceLookup, + * returns false. + * + * @param serviceIdentifier + * @return + */ + boolean isControllerServiceEnabling(String serviceIdentifier); + /** * Returns true if the given Controller Service is enabled, * false otherwise. If the given Controller Service is not @@ -63,4 +75,11 @@ public interface ControllerServiceLookup { */ Set getControllerServiceIdentifiers(Class serviceType) throws IllegalArgumentException; + /** + * Returns the name of the Controller service with the given identifier. If no service can be + * found with this identifier, returns {@code null}. + * @param serviceIdentifier + * @return + */ + String getControllerServiceName(String serviceIdentifier); } diff --git a/nifi/nifi-api/src/main/java/org/apache/nifi/logging/ComponentLog.java b/nifi/nifi-api/src/main/java/org/apache/nifi/logging/ComponentLog.java new file mode 100644 index 0000000000..c070e23698 --- /dev/null +++ b/nifi/nifi-api/src/main/java/org/apache/nifi/logging/ComponentLog.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.logging; + + +/** + *

      + * The ComponentLog provides a mechanism to ensure that all NiFi components are logging and reporting + * information in a consistent way. When messages are logged to the ComponentLog, each message has the + * following characteristics: + *

      + * + *
        + *
      • + * The toString() of the component is automatically prepended to the message so that it is clear + * which component is providing the information. This is important, since a single component may have many + * different instances within the same NiFi instance. + *
      • + *
      • + * If the last value in an Object[] argument that is passed to the logger is a Throwable, then the logged message + * will include a toString() of the Throwable; in addition, if the component's logger is set to + * DEBUG level via the logback configuration, the Stacktrace will also be logged. This provides a mechanism to easily + * enable stacktraces in the logs when they are desired without filling the logs with unneeded stack traces for messages + * that end up occurring often. + *
      • + *
      • + * Any message that is logged with a Severity level that meets or exceeds the configured Bulletin Level for that component + * will also cause a Bulletin to be generated, so that the message is visible in the UI, allowing Dataflow Managers + * to understand that a problem exists and what the issue is. + *
      • + *
      + * + */ +public interface ComponentLog { + void warn(String msg, Throwable t); + + void warn(String msg, Object[] os); + + void warn(String msg, Object[] os, Throwable t); + + void warn(String msg); + + void trace(String msg, Throwable t); + + void trace(String msg, Object[] os); + + void trace(String msg); + + void trace(String msg, Object[] os, Throwable t); + + boolean isWarnEnabled(); + + boolean isTraceEnabled(); + + boolean isInfoEnabled(); + + boolean isErrorEnabled(); + + boolean isDebugEnabled(); + + void info(String msg, Throwable t); + + void info(String msg, Object[] os); + + void info(String msg); + + void info(String msg, Object[] os, Throwable t); + + String getName(); + + void error(String msg, Throwable t); + + void error(String msg, Object[] os); + + void error(String msg); + + void error(String msg, Object[] os, Throwable t); + + void debug(String msg, Throwable t); + + void debug(String msg, Object[] os); + + void debug(String msg, Object[] os, Throwable t); + + void debug(String msg); +} diff --git a/nifi/nifi-api/src/main/java/org/apache/nifi/logging/ProcessorLog.java b/nifi/nifi-api/src/main/java/org/apache/nifi/logging/ProcessorLog.java index c5fa7b15ca..0d66d8553b 100644 --- a/nifi/nifi-api/src/main/java/org/apache/nifi/logging/ProcessorLog.java +++ b/nifi/nifi-api/src/main/java/org/apache/nifi/logging/ProcessorLog.java @@ -16,58 +16,15 @@ */ package org.apache.nifi.logging; -public interface ProcessorLog { - void warn(String msg, Throwable t); - - void warn(String msg, Object[] os); - - void warn(String msg, Object[] os, Throwable t); - - void warn(String msg); - - void trace(String msg, Throwable t); - - void trace(String msg, Object[] os); - - void trace(String msg); - - void trace(String msg, Object[] os, Throwable t); - - boolean isWarnEnabled(); - - boolean isTraceEnabled(); - - boolean isInfoEnabled(); - - boolean isErrorEnabled(); - - boolean isDebugEnabled(); - - void info(String msg, Throwable t); - - void info(String msg, Object[] os); - - void info(String msg); - - void info(String msg, Object[] os, Throwable t); - - String getName(); - - void error(String msg, Throwable t); - - void error(String msg, Object[] os); - - void error(String msg); - - void error(String msg, Object[] os, Throwable t); - - void debug(String msg, Throwable t); - - void debug(String msg, Object[] os); - - void debug(String msg, Object[] os, Throwable t); - - void debug(String msg); +/** + * The ProcessorLog is an extension of ComponentLog but provides no additional functionality. + * It exists because ProcessorLog was created first, + * but when Controller Services and Reporting Tasks began to be used more heavily loggers + * were needed for them as well. We did not want to return a ProcessorLog to a ControllerService + * or a ReportingTask, so all of the methods were moved to a higher interface named ComponentLog. + * However, we kept the ProcessorLog interface around in order to maintain backward compatibility. + */ +public interface ProcessorLog extends ComponentLog { } diff --git a/nifi/nifi-api/src/main/java/org/apache/nifi/reporting/AbstractReportingTask.java b/nifi/nifi-api/src/main/java/org/apache/nifi/reporting/AbstractReportingTask.java index 5ed8f24ac4..efcf2a3637 100644 --- a/nifi/nifi-api/src/main/java/org/apache/nifi/reporting/AbstractReportingTask.java +++ b/nifi/nifi-api/src/main/java/org/apache/nifi/reporting/AbstractReportingTask.java @@ -20,6 +20,7 @@ import java.util.concurrent.TimeUnit; import org.apache.nifi.components.AbstractConfigurableComponent; import org.apache.nifi.controller.ControllerServiceLookup; +import org.apache.nifi.logging.ComponentLog; import org.apache.nifi.processor.ProcessorInitializationContext; public abstract class AbstractReportingTask extends AbstractConfigurableComponent implements ReportingTask { @@ -28,10 +29,12 @@ public abstract class AbstractReportingTask extends AbstractConfigurableComponen private String name; private long schedulingNanos; private ControllerServiceLookup serviceLookup; + private ComponentLog logger; @Override public final void initialize(final ReportingInitializationContext config) throws InitializationException { identifier = config.getIdentifier(); + logger = config.getLogger(); name = config.getName(); schedulingNanos = config.getSchedulingPeriod(TimeUnit.NANOSECONDS); serviceLookup = config.getControllerServiceLookup(); @@ -91,4 +94,11 @@ public abstract class AbstractReportingTask extends AbstractConfigurableComponen protected void init(final ReportingInitializationContext config) throws InitializationException { } + /** + * Returns the logger that has been provided to the component by the framework in its initialize method. + * @return + */ + protected ComponentLog getLogger() { + return logger; + } } diff --git a/nifi/nifi-api/src/main/java/org/apache/nifi/reporting/ReportingInitializationContext.java b/nifi/nifi-api/src/main/java/org/apache/nifi/reporting/ReportingInitializationContext.java index a0ae88ea31..6b84589d75 100644 --- a/nifi/nifi-api/src/main/java/org/apache/nifi/reporting/ReportingInitializationContext.java +++ b/nifi/nifi-api/src/main/java/org/apache/nifi/reporting/ReportingInitializationContext.java @@ -19,6 +19,7 @@ package org.apache.nifi.reporting; import java.util.concurrent.TimeUnit; import org.apache.nifi.controller.ControllerServiceLookup; +import org.apache.nifi.logging.ComponentLog; import org.apache.nifi.scheduling.SchedulingStrategy; /** @@ -77,4 +78,13 @@ public interface ReportingInitializationContext { * @return */ SchedulingStrategy getSchedulingStrategy(); + + + /** + * Returns a logger that can be used to log important events in a standard way and generate + * bulletins when appropriate + * + * @return + */ + ComponentLog getLogger(); } diff --git a/nifi/nifi-api/src/main/java/org/apache/nifi/web/ClusterRequestException.java b/nifi/nifi-api/src/main/java/org/apache/nifi/web/ClusterRequestException.java index 0ecea3b56b..ee5f417d1f 100644 --- a/nifi/nifi-api/src/main/java/org/apache/nifi/web/ClusterRequestException.java +++ b/nifi/nifi-api/src/main/java/org/apache/nifi/web/ClusterRequestException.java @@ -17,6 +17,7 @@ package org.apache.nifi.web; /** + * An general error occurred when attempting to communicate with the cluster. */ public class ClusterRequestException extends RuntimeException { diff --git a/nifi/nifi-api/src/main/java/org/apache/nifi/web/ComponentDetails.java b/nifi/nifi-api/src/main/java/org/apache/nifi/web/ComponentDetails.java new file mode 100644 index 0000000000..0b68ed95ac --- /dev/null +++ b/nifi/nifi-api/src/main/java/org/apache/nifi/web/ComponentDetails.java @@ -0,0 +1,157 @@ +/* + * 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.web; + +import java.util.Collection; +import java.util.Map; + +/** + * Details about a given component. Contains configuration and current validation errors. + */ +public class ComponentDetails { + + private final String id; + private final String name; + private final String type; + private final String state; + private final String annotationData; + private final Map properties; + private final Collection validationErrors; + + private ComponentDetails(final Builder builder) { + this.id = builder.id; + this.name = builder.name; + this.type = builder.type; + this.state = builder.state; + this.annotationData = builder.annotationData; + this.properties = builder.properties; + this.validationErrors = builder.validationErrors; + } + + /** + * The component id. + * + * @return + */ + public String getId() { + return id; + } + + /** + * The component name. + * + * @return + */ + public String getName() { + return name; + } + + /** + * The component type. + * + * @return + */ + public String getType() { + return type; + } + + /** + * The component state. + * + * @return + */ + public String getState() { + return state; + } + + /** + * The component's annotation data. + * + * @return + */ + public String getAnnotationData() { + return annotationData; + } + + /** + * Mapping of component properties. + * + * @return + */ + public Map getProperties() { + return properties; + } + + /** + * Current validation errors for the component. + * + * @return + */ + public Collection getValidationErrors() { + return validationErrors; + } + + public static final class Builder { + + private String id; + private String name; + private String type; + private String state; + private String annotationData; + private Map properties; + private Collection validationErrors; + + public Builder id(final String id) { + this.id = id; + return this; + } + + public Builder name(final String name) { + this.name = name; + return this; + } + + public Builder type(final String type) { + this.type = type; + return this; + } + + public Builder state(final String state) { + this.state = state; + return this; + } + + public Builder annotationData(final String annotationData) { + this.annotationData = annotationData; + return this; + } + + public Builder properties(final Map properties) { + this.properties = properties; + return this; + } + + public Builder validateErrors(final Collection validationErrors) { + this.validationErrors = validationErrors; + return this; + } + + public ComponentDetails build() { + return new ComponentDetails(this); + } + } +} diff --git a/nifi/nifi-api/src/main/java/org/apache/nifi/web/ConfigurationAction.java b/nifi/nifi-api/src/main/java/org/apache/nifi/web/ConfigurationAction.java new file mode 100644 index 0000000000..066e77254c --- /dev/null +++ b/nifi/nifi-api/src/main/java/org/apache/nifi/web/ConfigurationAction.java @@ -0,0 +1,137 @@ +/* + * 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.web; + +/** + * An action that represents the configuration of a component. + */ +public class ConfigurationAction { + + private final String id; + private final String name; + private final String type; + private final String field; + private final String previousValue; + private final String value; + + private ConfigurationAction(final Builder builder) { + this.id = builder.id; + this.name = builder.name; + this.type = builder.type; + this.field = builder.field; + this.previousValue = builder.previousValue; + this.value = builder.value; + } + + /** + * The id of the component being modified. + * + * @return + */ + public String getId() { + return id; + } + + /** + * The name of the component being modified. + * + * @return + */ + public String getName() { + return name; + } + + /** + * The type of the component being modified. + * + * @return + */ + public String getType() { + return type; + } + + /** + * Gets the name of the field, property, etc that has been modified. + * + * @return + */ + public String getField() { + return field; + } + + /** + * Gets the previous value. + * + * @return + */ + public String getPreviousValue() { + return previousValue; + } + + /** + * Gets the new value. + * + * @return + */ + public String getValue() { + return value; + } + + public static class Builder { + + private String id; + private String name; + private String type; + private String field; + private String previousValue; + private String value; + + public Builder id(final String id) { + this.id = id; + return this; + } + + public Builder name(final String name) { + this.name = name; + return this; + } + + public Builder type(final String type) { + this.type = type; + return this; + } + + public Builder field(final String field) { + this.field = field; + return this; + } + + public Builder previousValue(final String previousValue) { + this.previousValue = previousValue; + return this; + } + + public Builder value(final String value) { + this.value = value; + return this; + } + + public ConfigurationAction build() { + return new ConfigurationAction(this); + } + } +} diff --git a/nifi/nifi-api/src/main/java/org/apache/nifi/web/NiFiWebConfigurationContext.java b/nifi/nifi-api/src/main/java/org/apache/nifi/web/NiFiWebConfigurationContext.java new file mode 100644 index 0000000000..50f0ca30f3 --- /dev/null +++ b/nifi/nifi-api/src/main/java/org/apache/nifi/web/NiFiWebConfigurationContext.java @@ -0,0 +1,102 @@ +/* + * 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.web; + +import java.util.Collection; + +import org.apache.nifi.controller.ControllerService; + +/** + * NiFi web context providing limited access to dataflow configuration for + * component custom UIs. + */ +public interface NiFiWebConfigurationContext { + + /** + * Gets the ControllerService for the specified identifier. If a + * corresponding service cannot be found, null is returned. If this NiFi is + * clustered, the only services available will be those those + * availability is NCM only. + * + * @param serviceIdentifier + * @return + */ + ControllerService getControllerService(String serviceIdentifier); + + /** + * Provides a mechanism for custom UIs to save actions to appear in NiFi + * configuration history. Note all fields within each Action must be + * populated. Null values will result in a failure to insert the audit + * record. Since the saving to these actions is separate from the actual + * configuration change, a failure to insert here will just generate a + * warning log message. The recording of these actions typically happens + * after a configuration change is applied. Since those changes have already + * been applied to the flow, we cannot revert them because of a failure to + * insert an audit record. + * + * @param requestContext + * @param actions + * @throws IllegalArgumentException When the requestContext isn't fully populated or + * isn't appropriate for the given request + */ + void saveActions(NiFiWebRequestContext requestContext, Collection actions); + + /** + * Gets the current user dn. Returns null if no user is found. + * + * @return + */ + String getCurrentUserDn(); + + /** + * Gets the current user name. Returns null if no user is found. + * + * @return + */ + String getCurrentUserName(); + + /** + * Sets the annotation data for the underlying component. + * + * @param configurationContext + * @param annotationData + * @return the configuration for the underlying component + * @throws ResourceNotFoundException if the underlying component does not exit + * @throws InvalidRevisionException if a revision other than the current + * revision is given + * @throws ClusterRequestException if the annotation data was unable to be + * set for the underlying component. This exception will only be thrown when operating + * in a cluster. + * @throws IllegalArgumentException When the requestContext isn't fully populated or + * isn't appropriate for the given request + */ + ComponentDetails setAnnotationData(NiFiWebConfigurationRequestContext configurationContext, String annotationData) throws ResourceNotFoundException, InvalidRevisionException, ClusterRequestException; + + /** + * Gets the details for the underlying component (including configuration, validation errors, and annotation data). + * + * @param requestContext + * @return the configuration for the underlying component + * @throws ResourceNotFoundException if the underlying component does not exit + * @throws ClusterRequestException if the underlying component was unable to be + * retrieved from the cluster. This exception will only be thrown when + * operating in a cluster. + * @throws IllegalArgumentException When the requestContext isn't fully populated or + * isn't appropriate for the given request + */ + ComponentDetails getComponentDetails(NiFiWebRequestContext requestContext) throws ResourceNotFoundException, ClusterRequestException; +} diff --git a/nifi/nifi-api/src/main/java/org/apache/nifi/web/NiFiWebConfigurationRequestContext.java b/nifi/nifi-api/src/main/java/org/apache/nifi/web/NiFiWebConfigurationRequestContext.java new file mode 100644 index 0000000000..791224122a --- /dev/null +++ b/nifi/nifi-api/src/main/java/org/apache/nifi/web/NiFiWebConfigurationRequestContext.java @@ -0,0 +1,31 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.web; + +/** + * Contextual details required to make a configuration request from a UI extension. + */ +public interface NiFiWebConfigurationRequestContext extends NiFiWebRequestContext { + + /** + * The revision to include in the request. + * + * @return the revision + */ + Revision getRevision(); + +} diff --git a/nifi/nifi-api/src/main/java/org/apache/nifi/web/NiFiWebContext.java b/nifi/nifi-api/src/main/java/org/apache/nifi/web/NiFiWebContext.java index 4c4f25d38f..01702addf8 100644 --- a/nifi/nifi-api/src/main/java/org/apache/nifi/web/NiFiWebContext.java +++ b/nifi/nifi-api/src/main/java/org/apache/nifi/web/NiFiWebContext.java @@ -24,6 +24,7 @@ import org.apache.nifi.controller.ControllerService; * NiFi web context providing limited access to dataflow configuration for * processor custom UIs. */ +@Deprecated public interface NiFiWebContext { /** diff --git a/nifi/nifi-api/src/main/java/org/apache/nifi/web/NiFiWebContextConfig.java b/nifi/nifi-api/src/main/java/org/apache/nifi/web/NiFiWebContextConfig.java index 808b9d66a9..2df94e4aa4 100644 --- a/nifi/nifi-api/src/main/java/org/apache/nifi/web/NiFiWebContextConfig.java +++ b/nifi/nifi-api/src/main/java/org/apache/nifi/web/NiFiWebContextConfig.java @@ -19,6 +19,7 @@ package org.apache.nifi.web; /** * Context configuration for methods invoked from the NiFiWebContext. */ +@Deprecated public interface NiFiWebContextConfig { /** diff --git a/nifi/nifi-api/src/main/java/org/apache/nifi/web/NiFiWebRequestContext.java b/nifi/nifi-api/src/main/java/org/apache/nifi/web/NiFiWebRequestContext.java new file mode 100644 index 0000000000..ac38221ae1 --- /dev/null +++ b/nifi/nifi-api/src/main/java/org/apache/nifi/web/NiFiWebRequestContext.java @@ -0,0 +1,58 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.web; + +/** + * Contextual details required to make a request from a UI extension. + */ +public interface NiFiWebRequestContext { + + /** + * Returns the type of UI extension is making the request. + * + * @return + */ + UiExtensionType getExtensionType(); + + /** + * The request protocol scheme (http or https). When scheme is https, the + * X509Certificate can be used for subsequent remote requests. + * + * @return the protocol scheme + */ + String getScheme(); + + /** + * The id of the component. + * + * @return the ID + */ + String getId(); + + /** + * Returns the proxied entities chain. The format of the chain is as + * follows: + * + * + * <CN=original-proxied-entity><CN=first-proxy><CN=second-proxy>... + * + * + * @return the proxied entities chain or null if no chain + */ + String getProxiedEntitiesChain(); + +} diff --git a/nifi/nifi-api/src/main/java/org/apache/nifi/web/ProcessorConfigurationAction.java b/nifi/nifi-api/src/main/java/org/apache/nifi/web/ProcessorConfigurationAction.java index 8385e4a239..ce5e069d30 100644 --- a/nifi/nifi-api/src/main/java/org/apache/nifi/web/ProcessorConfigurationAction.java +++ b/nifi/nifi-api/src/main/java/org/apache/nifi/web/ProcessorConfigurationAction.java @@ -19,6 +19,7 @@ package org.apache.nifi.web; /** * */ +@Deprecated public class ProcessorConfigurationAction { private final String processorId; diff --git a/nifi/nifi-api/src/main/java/org/apache/nifi/web/ProcessorInfo.java b/nifi/nifi-api/src/main/java/org/apache/nifi/web/ProcessorInfo.java index 04810981b6..e87e73e306 100644 --- a/nifi/nifi-api/src/main/java/org/apache/nifi/web/ProcessorInfo.java +++ b/nifi/nifi-api/src/main/java/org/apache/nifi/web/ProcessorInfo.java @@ -22,6 +22,7 @@ import java.util.Map; /** * */ +@Deprecated public class ProcessorInfo { private final String id; diff --git a/nifi/nifi-api/src/main/java/org/apache/nifi/web/Revision.java b/nifi/nifi-api/src/main/java/org/apache/nifi/web/Revision.java index 1881c2f526..8a6275e1c5 100644 --- a/nifi/nifi-api/src/main/java/org/apache/nifi/web/Revision.java +++ b/nifi/nifi-api/src/main/java/org/apache/nifi/web/Revision.java @@ -37,12 +37,12 @@ public class Revision implements Serializable { * the client ID */ private final String clientId; - + public Revision(Long revision, String clientId) { this.version = revision; this.clientId = clientId; } - + public String getClientId() { return clientId; } @@ -51,34 +51,6 @@ public class Revision implements Serializable { return version; } - /** - * A factory method for creating a new Revision instance whose version is - * this instance's version plus 1. - * - * @return an updated revision - */ - public Revision increment() { - final long incrementedVersion; - if (version == null) { - incrementedVersion = 0; - } else { - incrementedVersion = version + 1; - } - return new Revision(incrementedVersion, clientId); - } - - /** - * A factory method for creating a new Revision instance whose version is - * this instance's version plus 1 and whose client ID is the given client - * ID. - * - * @param clientId the client ID - * @return an updated revision - */ - public Revision increment(String clientId) { - return new Revision(increment().getVersion(), clientId); - } - @Override public boolean equals(final Object obj) { diff --git a/nifi/nifi-api/src/main/java/org/apache/nifi/web/UiExtensionType.java b/nifi/nifi-api/src/main/java/org/apache/nifi/web/UiExtensionType.java new file mode 100644 index 0000000000..0bbda16d66 --- /dev/null +++ b/nifi/nifi-api/src/main/java/org/apache/nifi/web/UiExtensionType.java @@ -0,0 +1,31 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.web; + +/** + * Types of UI extensions. Since a UI extension could support multiple + * types of custom UIs it will need to include the type so the framework + * can appropriate understand and process the request (recording actions + * in the audit database, replicating a request throughout the cluster to + * the appropriate endpoints, etc). + */ +public enum UiExtensionType { + ContentViewer, + ProcessorConfiguration, + ControllerServiceConfiguration, + ReportingTaskConfiguration +} diff --git a/nifi/nifi-assembly/pom.xml b/nifi/nifi-assembly/pom.xml index cae0f00a90..a26f214c87 100644 --- a/nifi/nifi-assembly/pom.xml +++ b/nifi/nifi-assembly/pom.xml @@ -187,8 +187,6 @@ ./conf/flow.xml.gz ./conf/archive/ - ./conf/reporting-tasks.xml - ./conf/controller-services.xml ./conf/authority-providers.xml ./conf/templates ./database_repository diff --git a/nifi/nifi-commons/nifi-processor-utilities/src/main/java/org/apache/nifi/processor/util/StandardValidators.java b/nifi/nifi-commons/nifi-processor-utilities/src/main/java/org/apache/nifi/processor/util/StandardValidators.java index a22e7bb249..d1621eda43 100644 --- a/nifi/nifi-commons/nifi-processor-utilities/src/main/java/org/apache/nifi/processor/util/StandardValidators.java +++ b/nifi/nifi-commons/nifi-processor-utilities/src/main/java/org/apache/nifi/processor/util/StandardValidators.java @@ -236,7 +236,7 @@ public class StandardValidators { @Override public ValidationResult validate(final String subject, final String input, final ValidationContext context) { if ( context.isExpressionLanguageSupported(subject) && context.isExpressionLanguagePresent(input) ) { - return new ValidationResult.Builder().subject(subject).input(input).explanation("Expression Language Present").build(); + return new ValidationResult.Builder().subject(subject).input(input).explanation("Expression Language Present").valid(true).build(); } try { @@ -254,7 +254,7 @@ public class StandardValidators { @Override public ValidationResult validate(final String subject, final String input, final ValidationContext context) { if ( context.isExpressionLanguageSupported(subject) && context.isExpressionLanguagePresent(input) ) { - return new ValidationResult.Builder().subject(subject).input(input).explanation("Expression Language Present").build(); + return new ValidationResult.Builder().subject(subject).input(input).explanation("Expression Language Present").valid(true).build(); } try { @@ -271,7 +271,7 @@ public class StandardValidators { @Override public ValidationResult validate(final String subject, final String input, final ValidationContext context) { if ( context.isExpressionLanguageSupported(subject) && context.isExpressionLanguagePresent(input) ) { - return new ValidationResult.Builder().subject(subject).input(input).explanation("Expression Language Present").build(); + return new ValidationResult.Builder().subject(subject).input(input).explanation("Expression Language Present").valid(true).build(); } if (input == null) { @@ -289,7 +289,7 @@ public class StandardValidators { @Override public ValidationResult validate(final String subject, final String input, final ValidationContext context) { if ( context.isExpressionLanguageSupported(subject) && context.isExpressionLanguagePresent(input) ) { - return new ValidationResult.Builder().subject(subject).input(input).explanation("Expression Language Present").build(); + return new ValidationResult.Builder().subject(subject).input(input).explanation("Expression Language Present").valid(true).build(); } if (input == null) { @@ -319,7 +319,7 @@ public class StandardValidators { @Override public ValidationResult validate(final String subject, final String input, final ValidationContext context) { if ( context.isExpressionLanguageSupported(subject) && context.isExpressionLanguagePresent(input) ) { - return new ValidationResult.Builder().subject(subject).input(input).explanation("Expression Language Present").build(); + return new ValidationResult.Builder().subject(subject).input(input).explanation("Expression Language Present").valid(true).build(); } try { @@ -347,7 +347,7 @@ public class StandardValidators { @Override public ValidationResult validate(final String subject, final String input, final ValidationContext context) { if ( context.isExpressionLanguageSupported(subject) && context.isExpressionLanguagePresent(input) ) { - return new ValidationResult.Builder().subject(subject).input(input).explanation("Expression Language Present").build(); + return new ValidationResult.Builder().subject(subject).input(input).explanation("Expression Language Present").valid(true).build(); } final ValidationResult vr = DATA_SIZE_VALIDATOR.validate(subject, input, context); @@ -372,7 +372,7 @@ public class StandardValidators { @Override public ValidationResult validate(final String subject, final String input, final ValidationContext context) { if ( context.isExpressionLanguageSupported(subject) && context.isExpressionLanguagePresent(input) ) { - return new ValidationResult.Builder().subject(subject).input(input).explanation("Expression Language Present").build(); + return new ValidationResult.Builder().subject(subject).input(input).explanation("Expression Language Present").valid(true).build(); } final boolean matches = pattern.matcher(input).matches(); @@ -457,7 +457,7 @@ public class StandardValidators { @Override public ValidationResult validate(final String subject, final String input, final ValidationContext context) { if ( context.isExpressionLanguageSupported(subject) && context.isExpressionLanguagePresent(input) ) { - return new ValidationResult.Builder().subject(subject).input(input).explanation("Expression Language Present").build(); + return new ValidationResult.Builder().subject(subject).input(input).explanation("Expression Language Present").valid(true).build(); } String reason = null; @@ -503,7 +503,7 @@ public class StandardValidators { @Override public ValidationResult validate(final String subject, final String input, final ValidationContext context) { if ( context.isExpressionLanguageSupported(subject) && context.isExpressionLanguagePresent(input) ) { - return new ValidationResult.Builder().subject(subject).input(input).explanation("Expression Language Present").build(); + return new ValidationResult.Builder().subject(subject).input(input).explanation("Expression Language Present").valid(true).build(); } if (input == null) { @@ -628,7 +628,7 @@ public class StandardValidators { @Override public ValidationResult validate(final String subject, final String input, final ValidationContext context) { if ( context.isExpressionLanguageSupported(subject) && context.isExpressionLanguagePresent(input) ) { - return new ValidationResult.Builder().subject(subject).input(input).explanation("Expression Language Present").build(); + return new ValidationResult.Builder().subject(subject).input(input).explanation("Expression Language Present").valid(true).build(); } final ControllerService svc = context.getControllerServiceLookup().getControllerService(input); diff --git a/nifi/nifi-commons/nifi-properties/src/main/java/org/apache/nifi/util/NiFiProperties.java b/nifi/nifi-commons/nifi-properties/src/main/java/org/apache/nifi/util/NiFiProperties.java index 87a82d40eb..3b427a77ae 100644 --- a/nifi/nifi-commons/nifi-properties/src/main/java/org/apache/nifi/util/NiFiProperties.java +++ b/nifi/nifi-commons/nifi-properties/src/main/java/org/apache/nifi/util/NiFiProperties.java @@ -42,8 +42,6 @@ public class NiFiProperties extends Properties { public static final String PROPERTIES_FILE_PATH = "nifi.properties.file.path"; public static final String FLOW_CONFIGURATION_FILE = "nifi.flow.configuration.file"; public static final String FLOW_CONFIGURATION_ARCHIVE_FILE = "nifi.flow.configuration.archive.file"; - public static final String TASK_CONFIGURATION_FILE = "nifi.reporting.task.configuration.file"; - public static final String SERVICE_CONFIGURATION_FILE = "nifi.controller.service.configuration.file"; public static final String AUTHORITY_PROVIDER_CONFIGURATION_FILE = "nifi.authority.provider.configuration.file"; public static final String REPOSITORY_DATABASE_DIRECTORY = "nifi.database.directory"; public static final String RESTORE_DIRECTORY = "nifi.restore.directory"; diff --git a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/socket/EndpointConnectionPool.java b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/socket/EndpointConnectionPool.java index e80f3284f5..daf52b4f07 100644 --- a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/socket/EndpointConnectionPool.java +++ b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/socket/EndpointConnectionPool.java @@ -700,7 +700,7 @@ public class EndpointConnectionPool { final int flowFileCount = nodeInfo.getTotalFlowFiles(); // don't allow any node to get more than 80% of the data final double percentageOfFlowFiles = Math.min(0.8D, ((double) flowFileCount / (double) totalFlowFileCount)); - final double relativeWeighting = (direction == TransferDirection.RECEIVE) ? (1 - percentageOfFlowFiles) : percentageOfFlowFiles; + final double relativeWeighting = (direction == TransferDirection.SEND) ? (1 - percentageOfFlowFiles) : percentageOfFlowFiles; final int entries = Math.max(1, (int) (numDestinations * relativeWeighting)); entryCountMap.put(nodeInfo, Math.max(1, entries)); diff --git a/nifi/nifi-commons/nifi-site-to-site-client/src/test/java/org/apache/nifi/remote/client/socket/TestEndpointConnectionStatePool.java b/nifi/nifi-commons/nifi-site-to-site-client/src/test/java/org/apache/nifi/remote/client/socket/TestEndpointConnectionStatePool.java index cb7af08e84..c5cca78614 100644 --- a/nifi/nifi-commons/nifi-site-to-site-client/src/test/java/org/apache/nifi/remote/client/socket/TestEndpointConnectionStatePool.java +++ b/nifi/nifi-commons/nifi-site-to-site-client/src/test/java/org/apache/nifi/remote/client/socket/TestEndpointConnectionStatePool.java @@ -39,7 +39,7 @@ public class TestEndpointConnectionStatePool { collection.add(new NodeInformation("ShouldGetMedium", 5, 5555, true, 4096)); clusterNodeInfo.setNodeInformation(collection); - final List destinations = EndpointConnectionPool.formulateDestinationList(clusterNodeInfo, TransferDirection.SEND); + final List destinations = EndpointConnectionPool.formulateDestinationList(clusterNodeInfo, TransferDirection.RECEIVE); for ( final PeerStatus peerStatus : destinations ) { System.out.println(peerStatus.getPeerDescription()); } @@ -53,7 +53,7 @@ public class TestEndpointConnectionStatePool { collection.add(new NodeInformation("ShouldGetLots", 2, 2222, true, 50000)); clusterNodeInfo.setNodeInformation(collection); - final List destinations = EndpointConnectionPool.formulateDestinationList(clusterNodeInfo, TransferDirection.SEND); + final List destinations = EndpointConnectionPool.formulateDestinationList(clusterNodeInfo, TransferDirection.RECEIVE); for ( final PeerStatus peerStatus : destinations ) { System.out.println(peerStatus.getPeerDescription()); } diff --git a/nifi/nifi-mock/src/main/java/org/apache/nifi/util/MockControllerServiceInitializationContext.java b/nifi/nifi-mock/src/main/java/org/apache/nifi/util/MockControllerServiceInitializationContext.java index 86624aeaaa..fd3c2de562 100644 --- a/nifi/nifi-mock/src/main/java/org/apache/nifi/util/MockControllerServiceInitializationContext.java +++ b/nifi/nifi-mock/src/main/java/org/apache/nifi/util/MockControllerServiceInitializationContext.java @@ -19,13 +19,20 @@ package org.apache.nifi.util; import org.apache.nifi.controller.ControllerService; import org.apache.nifi.controller.ControllerServiceInitializationContext; import org.apache.nifi.controller.ControllerServiceLookup; +import org.apache.nifi.logging.ComponentLog; public class MockControllerServiceInitializationContext extends MockControllerServiceLookup implements ControllerServiceInitializationContext, ControllerServiceLookup { private final String identifier; + private final ComponentLog logger; public MockControllerServiceInitializationContext(final ControllerService controllerService, final String identifier) { + this(controllerService, identifier, new MockProcessorLog(identifier, controllerService)); + } + + public MockControllerServiceInitializationContext(final ControllerService controllerService, final String identifier, final ComponentLog logger) { this.identifier = identifier; + this.logger = logger; addControllerService(controllerService, identifier); } @@ -33,9 +40,19 @@ public class MockControllerServiceInitializationContext extends MockControllerSe public String getIdentifier() { return identifier; } + + @Override + public String getControllerServiceName(final String serviceIdentifier) { + return null; + } @Override public ControllerServiceLookup getControllerServiceLookup() { return this; } + + @Override + public ComponentLog getLogger() { + return logger; + } } diff --git a/nifi/nifi-mock/src/main/java/org/apache/nifi/util/MockControllerServiceLookup.java b/nifi/nifi-mock/src/main/java/org/apache/nifi/util/MockControllerServiceLookup.java index 8298a39617..2734440497 100644 --- a/nifi/nifi-mock/src/main/java/org/apache/nifi/util/MockControllerServiceLookup.java +++ b/nifi/nifi-mock/src/main/java/org/apache/nifi/util/MockControllerServiceLookup.java @@ -76,6 +76,11 @@ public abstract class MockControllerServiceLookup implements ControllerServiceLo return isControllerServiceEnabled(service.getIdentifier()); } + @Override + public boolean isControllerServiceEnabling(final String serviceIdentifier) { + return false; + } + @Override public Set getControllerServiceIdentifiers(final Class serviceType) { final Set ids = new HashSet<>(); @@ -86,4 +91,10 @@ public abstract class MockControllerServiceLookup implements ControllerServiceLo } return ids; } + + @Override + public String getControllerServiceName(String serviceIdentifier) { + final ControllerServiceConfiguration status = controllerServiceMap.get(serviceIdentifier); + return status == null ? null : serviceIdentifier; + } } diff --git a/nifi/nifi-mock/src/main/java/org/apache/nifi/util/MockProcessorInitializationContext.java b/nifi/nifi-mock/src/main/java/org/apache/nifi/util/MockProcessorInitializationContext.java index f49a6c5ee9..0aa2749a57 100644 --- a/nifi/nifi-mock/src/main/java/org/apache/nifi/util/MockProcessorInitializationContext.java +++ b/nifi/nifi-mock/src/main/java/org/apache/nifi/util/MockProcessorInitializationContext.java @@ -62,6 +62,11 @@ public class MockProcessorInitializationContext implements ProcessorInitializati return this; } + @Override + public String getControllerServiceName(String serviceIdentifier) { + return context.getControllerServiceName(serviceIdentifier); + } + @Override public boolean isControllerServiceEnabled(String serviceIdentifier) { return context.isControllerServiceEnabled(serviceIdentifier); @@ -71,4 +76,9 @@ public class MockProcessorInitializationContext implements ProcessorInitializati public boolean isControllerServiceEnabled(ControllerService service) { return context.isControllerServiceEnabled(service); } + + @Override + public boolean isControllerServiceEnabling(String serviceIdentifier) { + return context.isControllerServiceEnabling(serviceIdentifier); + } } diff --git a/nifi/nifi-mock/src/main/java/org/apache/nifi/util/MockProcessorLog.java b/nifi/nifi-mock/src/main/java/org/apache/nifi/util/MockProcessorLog.java index f8489f88ea..5505e88fec 100644 --- a/nifi/nifi-mock/src/main/java/org/apache/nifi/util/MockProcessorLog.java +++ b/nifi/nifi-mock/src/main/java/org/apache/nifi/util/MockProcessorLog.java @@ -17,28 +17,26 @@ package org.apache.nifi.util; import org.apache.nifi.logging.ProcessorLog; -import org.apache.nifi.processor.Processor; - import org.slf4j.Logger; import org.slf4j.LoggerFactory; public class MockProcessorLog implements ProcessorLog { private final Logger logger; - private final Processor processor; + private final Object component; - public MockProcessorLog(final String processorId, final Processor processor) { - this.logger = LoggerFactory.getLogger(processor.getClass()); - this.processor = processor; + public MockProcessorLog(final String componentId, final Object component) { + this.logger = LoggerFactory.getLogger(component.getClass()); + this.component = component; } private Object[] addProcessor(final Object[] originalArgs) { - return prependToArgs(originalArgs, processor); + return prependToArgs(originalArgs, component); } private Object[] addProcessorAndThrowable(final Object[] os, final Throwable t) { final Object[] modifiedArgs = new Object[os.length + 2]; - modifiedArgs[0] = processor.toString(); + modifiedArgs[0] = component.toString(); for (int i = 0; i < os.length; i++) { modifiedArgs[i + 1] = os[i]; } @@ -75,7 +73,7 @@ public class MockProcessorLog implements ProcessorLog { */ @Override public void warn(final String msg, final Throwable t) { - warn("{} " + msg, new Object[]{processor}, t); + warn("{} " + msg, new Object[]{component}, t); } /** @@ -118,7 +116,7 @@ public class MockProcessorLog implements ProcessorLog { @Override public void warn(String msg) { msg = "{} " + msg; - logger.warn(msg, processor); + logger.warn(msg, component); } /** @@ -129,7 +127,7 @@ public class MockProcessorLog implements ProcessorLog { @Override public void trace(String msg, Throwable t) { msg = "{} " + msg; - final Object[] os = {processor}; + final Object[] os = {component}; logger.trace(msg, os, t); } @@ -152,7 +150,7 @@ public class MockProcessorLog implements ProcessorLog { @Override public void trace(String msg) { msg = "{} " + msg; - final Object[] os = {processor}; + final Object[] os = {component}; logger.trace(msg, os); } @@ -224,7 +222,7 @@ public class MockProcessorLog implements ProcessorLog { @Override public void info(String msg, Throwable t) { msg = "{} " + msg; - final Object[] os = {processor}; + final Object[] os = {component}; logger.info(msg, os); if (logger.isDebugEnabled()) { @@ -252,7 +250,7 @@ public class MockProcessorLog implements ProcessorLog { @Override public void info(String msg) { msg = "{} " + msg; - final Object[] os = {processor}; + final Object[] os = {component}; logger.info(msg, os); } @@ -291,7 +289,7 @@ public class MockProcessorLog implements ProcessorLog { @Override public void error(String msg, Throwable t) { msg = "{} " + msg; - final Object[] os = {processor}; + final Object[] os = {component}; logger.error(msg, os, t); if (logger.isDebugEnabled()) { @@ -322,7 +320,7 @@ public class MockProcessorLog implements ProcessorLog { @Override public void error(String msg) { msg = "{} " + msg; - final Object[] os = {processor}; + final Object[] os = {component}; logger.error(msg, os); } @@ -352,7 +350,7 @@ public class MockProcessorLog implements ProcessorLog { @Override public void debug(String msg, Throwable t) { msg = "{} " + msg; - final Object[] os = {processor}; + final Object[] os = {component}; logger.debug(msg, os, t); } @@ -394,7 +392,7 @@ public class MockProcessorLog implements ProcessorLog { @Override public void debug(String msg) { msg = "{} " + msg; - final Object[] os = {processor}; + final Object[] os = {component}; logger.debug(msg, os); } diff --git a/nifi/nifi-mock/src/main/java/org/apache/nifi/util/MockReportingInitializationContext.java b/nifi/nifi-mock/src/main/java/org/apache/nifi/util/MockReportingInitializationContext.java index a874536d67..7cabef22a7 100644 --- a/nifi/nifi-mock/src/main/java/org/apache/nifi/util/MockReportingInitializationContext.java +++ b/nifi/nifi-mock/src/main/java/org/apache/nifi/util/MockReportingInitializationContext.java @@ -22,6 +22,7 @@ import java.util.concurrent.TimeUnit; import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.controller.ControllerServiceLookup; +import org.apache.nifi.logging.ComponentLog; import org.apache.nifi.reporting.ReportingInitializationContext; import org.apache.nifi.scheduling.SchedulingStrategy; @@ -30,10 +31,12 @@ public class MockReportingInitializationContext extends MockControllerServiceLoo private final String identifier; private final String name; private final Map properties = new HashMap<>(); + private final ComponentLog logger; - public MockReportingInitializationContext(final String identifier, final String name) { + public MockReportingInitializationContext(final String identifier, final String name, final ComponentLog logger) { this.identifier = identifier; this.name = name; + this.logger = logger; } @Override @@ -78,4 +81,9 @@ public class MockReportingInitializationContext extends MockControllerServiceLoo public SchedulingStrategy getSchedulingStrategy() { return SchedulingStrategy.TIMER_DRIVEN; } + + @Override + public ComponentLog getLogger() { + return logger; + } } diff --git a/nifi/nifi-mock/src/main/java/org/apache/nifi/util/MockValidationContext.java b/nifi/nifi-mock/src/main/java/org/apache/nifi/util/MockValidationContext.java index c00386ef5d..c9b1cdaf30 100644 --- a/nifi/nifi-mock/src/main/java/org/apache/nifi/util/MockValidationContext.java +++ b/nifi/nifi-mock/src/main/java/org/apache/nifi/util/MockValidationContext.java @@ -103,6 +103,21 @@ public class MockValidationContext implements ValidationContext, ControllerServi } @Override + public String getControllerServiceName(final String serviceIdentifier) { + final ControllerServiceConfiguration configuration = context.getConfiguration(serviceIdentifier); + return configuration == null ? null : serviceIdentifier; + } + + @Override + public boolean isValidationRequired(final ControllerService service) { + return true; + } + + @Override + public boolean isControllerServiceEnabling(String serviceIdentifier) { + return context.isControllerServiceEnabling(serviceIdentifier); + } + public boolean isExpressionLanguagePresent(final String value) { if ( value == null ) { return false; diff --git a/nifi/nifi-mock/src/main/java/org/apache/nifi/util/StandardProcessorTestRunner.java b/nifi/nifi-mock/src/main/java/org/apache/nifi/util/StandardProcessorTestRunner.java index 8d691dd879..d66ed8165d 100644 --- a/nifi/nifi-mock/src/main/java/org/apache/nifi/util/StandardProcessorTestRunner.java +++ b/nifi/nifi-mock/src/main/java/org/apache/nifi/util/StandardProcessorTestRunner.java @@ -59,6 +59,7 @@ import org.apache.nifi.controller.ConfigurationContext; import org.apache.nifi.controller.ControllerService; import org.apache.nifi.flowfile.FlowFile; import org.apache.nifi.flowfile.attributes.CoreAttributes; +import org.apache.nifi.logging.ComponentLog; import org.apache.nifi.processor.ProcessSessionFactory; import org.apache.nifi.processor.Processor; import org.apache.nifi.processor.QueueSize; @@ -512,13 +513,15 @@ public class StandardProcessorTestRunner implements TestRunner { @Override public void addControllerService(final String identifier, final ControllerService service, final Map properties) throws InitializationException { + // hold off on failing due to deprecated annotation for now... will introduce later. // for ( final Method method : service.getClass().getMethods() ) { // if ( method.isAnnotationPresent(org.apache.nifi.controller.annotation.OnConfigured.class) ) { // Assert.fail("Controller Service " + service + " is using deprecated Annotation " + org.apache.nifi.controller.annotation.OnConfigured.class + " for method " + method); // } // } - final MockControllerServiceInitializationContext initContext = new MockControllerServiceInitializationContext(requireNonNull(service), requireNonNull(identifier)); + final ComponentLog logger = new MockProcessorLog(identifier, service); + final MockControllerServiceInitializationContext initContext = new MockControllerServiceInitializationContext(requireNonNull(service), requireNonNull(identifier), logger); service.initialize(initContext); final Map resolvedProps = new HashMap<>(); diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/dao/ActionDAO.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/dao/ActionDAO.java index 5d6d2226d8..5d5d498ee5 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/dao/ActionDAO.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/dao/ActionDAO.java @@ -48,12 +48,12 @@ public interface ActionDAO { /** * Finds the previous values for the specified property in the specified - * processor. Returns empty list if there are none. + * component. Returns empty list if there are none. * - * @param processorId + * @param componentId * @return */ - Map> getPreviousValues(String processorId); + Map> getPreviousValues(String componentId); /** * Finds the specified action. diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/dao/impl/StandardActionDAO.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/dao/impl/StandardActionDAO.java index 4b89655c47..34623f4248 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/dao/impl/StandardActionDAO.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/dao/impl/StandardActionDAO.java @@ -32,7 +32,7 @@ import org.apache.nifi.action.Action; import org.apache.nifi.action.Component; import org.apache.nifi.action.Operation; import org.apache.nifi.action.component.details.ComponentDetails; -import org.apache.nifi.action.component.details.ProcessorDetails; +import org.apache.nifi.action.component.details.ExtensionDetails; import org.apache.nifi.action.component.details.RemoteProcessGroupDetails; import org.apache.nifi.action.details.ActionDetails; import org.apache.nifi.action.details.ConfigureDetails; @@ -70,7 +70,7 @@ public class StandardActionDAO implements ActionDAO { // ----------------- // component details // ----------------- - private static final String INSERT_PROCESSOR_DETAILS = "INSERT INTO PROCESSOR_DETAILS (" + private static final String INSERT_EXTENSION_DETAILS = "INSERT INTO PROCESSOR_DETAILS (" + "ACTION_ID, TYPE" + ") VALUES (" + "?, " @@ -145,7 +145,7 @@ public class StandardActionDAO implements ActionDAO { // ----------------- // component details // ----------------- - private static final String SELECT_PROCESSOR_DETAILS_FOR_ACTION = "SELECT * FROM PROCESSOR_DETAILS WHERE ACTION_ID = ?"; + private static final String SELECT_EXTENSION_DETAILS_FOR_ACTION = "SELECT * FROM PROCESSOR_DETAILS WHERE ACTION_ID = ?"; private static final String SELECT_REMOTE_PROCESS_GROUP_DETAILS_FOR_ACTION = "SELECT * FROM REMOTE_PROCESS_GROUP_DETAILS WHERE ACTION_ID = ?"; @@ -179,8 +179,8 @@ public class StandardActionDAO implements ActionDAO { + "ORDER BY A.ACTION_TIMESTAMP DESC " + "LIMIT 4"; - private Connection connection; - private Map columnMap; + private final Connection connection; + private final Map columnMap; public StandardActionDAO(Connection connection) { this.connection = connection; @@ -233,8 +233,8 @@ public class StandardActionDAO implements ActionDAO { // determine the type of component ComponentDetails componentDetails = action.getComponentDetails(); - if (componentDetails instanceof ProcessorDetails) { - createProcessorDetails(action.getId(), (ProcessorDetails) componentDetails); + if (componentDetails instanceof ExtensionDetails) { + createExtensionDetails(action.getId(), (ExtensionDetails) componentDetails); } else if (componentDetails instanceof RemoteProcessGroupDetails) { createRemoteProcessGroupDetails(action.getId(), (RemoteProcessGroupDetails) componentDetails); } @@ -260,26 +260,26 @@ public class StandardActionDAO implements ActionDAO { } /** - * Persists the processor details. + * Persists the extension details. * * @param actionId - * @param processorDetails + * @param extensionDetails * @throws DataAccessException */ - private void createProcessorDetails(int actionId, ProcessorDetails processorDetails) throws DataAccessException { + private void createExtensionDetails(int actionId, ExtensionDetails extensionDetails) throws DataAccessException { PreparedStatement statement = null; try { - // obtain a statement to insert to the processor action table - statement = connection.prepareStatement(INSERT_PROCESSOR_DETAILS); + // obtain a statement to insert to the extension action table + statement = connection.prepareStatement(INSERT_EXTENSION_DETAILS); statement.setInt(1, actionId); - statement.setString(2, StringUtils.left(processorDetails.getType(), 1000)); + statement.setString(2, StringUtils.left(extensionDetails.getType(), 1000)); // insert the action int updateCount = statement.executeUpdate(); // ensure the operation completed successfully if (updateCount != 1) { - throw new DataAccessException("Unable to insert processor details."); + throw new DataAccessException("Unable to insert extension details."); } } catch (SQLException sqle) { throw new DataAccessException(sqle); @@ -601,8 +601,8 @@ public class StandardActionDAO implements ActionDAO { // get the component details if appropriate ComponentDetails componentDetails = null; - if (Component.Processor.equals(component)) { - componentDetails = getProcessorDetails(actionId); + if (Component.Processor.equals(component) || Component.ControllerService.equals(component) || Component.ReportingTask.equals(component)) { + componentDetails = getExtensionDetails(actionId); } else if (Component.RemoteProcessGroup.equals(component)) { componentDetails = getRemoteProcessGroupDetails(actionId); } @@ -675,8 +675,8 @@ public class StandardActionDAO implements ActionDAO { // get the component details if appropriate ComponentDetails componentDetails = null; - if (Component.Processor.equals(component)) { - componentDetails = getProcessorDetails(actionId); + if (Component.Processor.equals(component) || Component.ControllerService.equals(component) || Component.ReportingTask.equals(component)) { + componentDetails = getExtensionDetails(actionId); } else if (Component.RemoteProcessGroup.equals(component)) { componentDetails = getRemoteProcessGroupDetails(actionId); } @@ -713,19 +713,19 @@ public class StandardActionDAO implements ActionDAO { } /** - * Loads the specified processor details. + * Loads the specified extension details. * * @param actionId * @return * @throws DataAccessException */ - private ProcessorDetails getProcessorDetails(Integer actionId) throws DataAccessException { - ProcessorDetails processorDetails = null; + private ExtensionDetails getExtensionDetails(Integer actionId) throws DataAccessException { + ExtensionDetails extensionDetails = null; PreparedStatement statement = null; ResultSet rs = null; try { // create the statement - statement = connection.prepareStatement(SELECT_PROCESSOR_DETAILS_FOR_ACTION); + statement = connection.prepareStatement(SELECT_EXTENSION_DETAILS_FOR_ACTION); statement.setInt(1, actionId); // execute the query @@ -733,8 +733,8 @@ public class StandardActionDAO implements ActionDAO { // ensure results if (rs.next()) { - processorDetails = new ProcessorDetails(); - processorDetails.setType(rs.getString("TYPE")); + extensionDetails = new ExtensionDetails(); + extensionDetails.setType(rs.getString("TYPE")); } } catch (SQLException sqle) { throw new DataAccessException(sqle); @@ -743,7 +743,7 @@ public class StandardActionDAO implements ActionDAO { RepositoryUtils.closeQuietly(statement); } - return processorDetails; + return extensionDetails; } /** @@ -931,7 +931,7 @@ public class StandardActionDAO implements ActionDAO { } @Override - public Map> getPreviousValues(String processorId) { + public Map> getPreviousValues(String componentId) { Map> previousValues = new LinkedHashMap<>(); PreparedStatement statement = null; @@ -939,7 +939,7 @@ public class StandardActionDAO implements ActionDAO { try { // create the statement statement = connection.prepareStatement(SELECT_PREVIOUSLY_CONFIGURED_FIELDS); - statement.setString(1, processorId); + statement.setString(1, componentId); // execute the query rs = statement.executeQuery(); @@ -947,7 +947,7 @@ public class StandardActionDAO implements ActionDAO { // ensure results while (rs.next()) { final String property = rs.getString("NAME"); - previousValues.put(property, getPreviousValuesForProperty(processorId, property)); + previousValues.put(property, getPreviousValuesForProperty(componentId, property)); } } catch (SQLException sqle) { throw new DataAccessException(sqle); @@ -959,7 +959,7 @@ public class StandardActionDAO implements ActionDAO { return previousValues; } - private List getPreviousValuesForProperty(final String processorId, final String property) { + private List getPreviousValuesForProperty(final String componentId, final String property) { List previousValues = new ArrayList<>(); PreparedStatement statement = null; @@ -967,7 +967,7 @@ public class StandardActionDAO implements ActionDAO { try { // create the statement statement = connection.prepareStatement(SELECT_PREVIOUS_VALUES); - statement.setString(1, processorId); + statement.setString(1, componentId); statement.setString(2, property); // execute the query diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/AuditService.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/AuditService.java index 0843bd804a..7ca4e0618a 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/AuditService.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/AuditService.java @@ -40,12 +40,12 @@ public interface AuditService { /** * Finds the previous values for the specified property in the specified - * processor. Returns null if there are none. + * component. Returns null if there are none. * - * @param processorId + * @param componentId * @return */ - Map> getPreviousValues(String processorId); + Map> getPreviousValues(String componentId); /** * Get the actions within the given date range. diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/action/GetPreviousValues.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/action/GetPreviousValues.java index 5ce663eb39..569439baa9 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/action/GetPreviousValues.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/action/GetPreviousValues.java @@ -28,16 +28,16 @@ import org.apache.nifi.history.PreviousValue; */ public class GetPreviousValues implements AdministrationAction>> { - private final String processorId; + private final String componentId; - public GetPreviousValues(String processorId) { - this.processorId = processorId; + public GetPreviousValues(String componentId) { + this.componentId = componentId; } @Override public Map> execute(DAOFactory daoFactory, AuthorityProvider authorityProvider) { ActionDAO actionDao = daoFactory.getActionDAO(); - return actionDao.getPreviousValues(processorId); + return actionDao.getPreviousValues(componentId); } } diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/impl/StandardAuditService.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/impl/StandardAuditService.java index 127f1df813..721e6b27ec 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/impl/StandardAuditService.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/service/impl/StandardAuditService.java @@ -81,7 +81,7 @@ public class StandardAuditService implements AuditService { } @Override - public Map> getPreviousValues(String processorId) { + public Map> getPreviousValues(String componentId) { Transaction transaction = null; Map> previousValues = null; @@ -91,7 +91,7 @@ public class StandardAuditService implements AuditService { transaction = transactionBuilder.start(); // seed the accounts - GetPreviousValues getActions = new GetPreviousValues(processorId); + GetPreviousValues getActions = new GetPreviousValues(componentId); previousValues = transaction.execute(getActions); // commit the transaction diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ProcessorHistoryDTO.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ComponentHistoryDTO.java similarity index 77% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ProcessorHistoryDTO.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ComponentHistoryDTO.java index 2741116a54..3bdbe2814e 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ProcessorHistoryDTO.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ComponentHistoryDTO.java @@ -20,29 +20,29 @@ import java.util.Map; import javax.xml.bind.annotation.XmlType; /** - * History of a processor's properties. + * History of a component's properties. */ -@XmlType(name = "processorHistory") -public class ProcessorHistoryDTO { +@XmlType(name = "componentHistory") +public class ComponentHistoryDTO { - private String processorId; + private String componentId; private Map propertyHistory; /** - * The processor id. + * The component id. * * @return */ - public String getProcessorId() { - return processorId; + public String getComponentId() { + return componentId; } - public void setProcessorId(String processorId) { - this.processorId = processorId; + public void setComponentId(String componentId) { + this.componentId = componentId; } /** - * The history for this processors properties. + * The history for this components properties. * * @return */ diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ControllerConfigurationDTO.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ControllerConfigurationDTO.java index b916025fd1..190cb47fe2 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ControllerConfigurationDTO.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ControllerConfigurationDTO.java @@ -16,7 +16,10 @@ */ package org.apache.nifi.web.api.dto; +import java.util.Date; import javax.xml.bind.annotation.XmlType; +import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter; +import org.apache.nifi.web.api.dto.util.TimeAdapter; /** * Details for the controller configuration. @@ -32,6 +35,7 @@ public class ControllerConfigurationDTO { private Long autoRefreshIntervalSeconds; private Boolean siteToSiteSecure; + private Date currentTime; private Integer timeOffset; private String contentViewerUrl; @@ -117,6 +121,20 @@ public class ControllerConfigurationDTO { this.siteToSiteSecure = siteToSiteSecure; } + /** + * The current time on the server. + * + * @return + */ + @XmlJavaTypeAdapter(TimeAdapter.class) + public Date getCurrentTime() { + return currentTime; + } + + public void setCurrentTime(Date currentTime) { + this.currentTime = currentTime; + } + /** * The time offset of the server. * diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ControllerServiceDTO.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ControllerServiceDTO.java new file mode 100644 index 0000000000..75d18a2caa --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ControllerServiceDTO.java @@ -0,0 +1,190 @@ +/* + * 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.web.api.dto; + +import java.util.Collection; +import java.util.Map; +import java.util.Set; +import javax.xml.bind.annotation.XmlType; + +/** + * A Controller Service that can be shared by other components + */ +@XmlType(name = "controllerService") +public class ControllerServiceDTO extends NiFiComponentDTO { + + private String name; + private String type; + private String comments; + private String availability; + private String state; + + private Map properties; + private Map descriptors; + + private String customUiUrl; + private String annotationData; + + private Set referencingComponents; + + private Collection validationErrors; + + /** + * The controller service name. + * + * @return + */ + public String getName() { + return name; + } + + public void setName(String name) { + this.name = name; + } + + /** + * The controller service type. + * + * @return + */ + public String getType() { + return type; + } + + public void setType(String type) { + this.type = type; + } + + + /** + * The comment for the Controller Service + * @return + */ + public String getComments() { + return comments; + } + + public void setComments(String comments) { + this.comments = comments; + } + + /** + * Where this service is available. Possible values are NCM, NODE. + * + * @return + */ + public String getAvailability() { + return availability; + } + + public void setAvailability(String availability) { + this.availability = availability; + } + + /** + * The state of this controller service. Possible values are ENABLED, ENABLING, DISABLED, DISABLING. + * @return + */ + public String getState() { + return state; + } + + public void setState(String state) { + this.state = state; + } + + /** + * The controller service properties. + * + * @return + */ + public Map getProperties() { + return properties; + } + + public void setProperties(Map properties) { + this.properties = properties; + } + + /** + * The descriptors for the controller service properties. + * + * @return + */ + public Map getDescriptors() { + return descriptors; + } + + public void setDescriptors(Map descriptors) { + this.descriptors = descriptors; + } + + /** + * Returns the URL for this controller services custom configuration UI + * if applicable. Null otherwise. + * + * @return + */ + public String getCustomUiUrl() { + return customUiUrl; + } + + public void setCustomUiUrl(String customUiUrl) { + this.customUiUrl = customUiUrl; + } + + /** + * The annotation data for this controller service. + * + * @return + */ + public String getAnnotationData() { + return annotationData; + } + + public void setAnnotationData(String annotationData) { + this.annotationData = annotationData; + } + + /** + * All components referencing this controller service. + * + * @return + */ + public Set getReferencingComponents() { + return referencingComponents; + } + + public void setReferencingComponents(Set referencingComponents) { + this.referencingComponents = referencingComponents; + } + + /** + * Gets the validation errors from this controller service. These validation errors + * represent the problems with the controller service that must be resolved before it + * can be enabled. + * + * @return The validation errors + */ + public Collection getValidationErrors() { + return validationErrors; + } + + public void setValidationErrors(Collection validationErrors) { + this.validationErrors = validationErrors; + } +} diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ControllerServiceReferencingComponentDTO.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ControllerServiceReferencingComponentDTO.java new file mode 100644 index 0000000000..7fc57ffc0b --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ControllerServiceReferencingComponentDTO.java @@ -0,0 +1,207 @@ +/* + * 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.web.api.dto; + +import java.util.Collection; +import java.util.Map; +import java.util.Set; +import javax.xml.bind.annotation.XmlType; + +/** + * A component referencing a controller service. This can either be another + * controller service or a processor. Depending on the type of component + * different properties may be set. + */ +@XmlType(name = "controllerServiceReferencingComponent") +public class ControllerServiceReferencingComponentDTO { + private String groupId; + private String id; + private String name; + private String type; + private String state; + + private Map properties; + private Map descriptors; + + private Collection validationErrors; + + private String referenceType; + private Integer activeThreadCount; + + private Boolean referenceCycle; + private Set referencingComponents; + + /** + * Group id for this component referencing a controller service. If this + * component is another service, this field is blank. + * + * @return + */ + public String getGroupId() { + return groupId; + } + + public void setGroupId(String groupId) { + this.groupId = groupId; + } + + /** + * The id for this component referencing a controller service. + * + * @return + */ + public String getId() { + return id; + } + + public void setId(String id) { + this.id = id; + } + + /** + * The name for this component referencing a controller service. + * + * @return + */ + public String getName() { + return name; + } + + public void setName(String name) { + this.name = name; + } + + /** + * The type for this component referencing a controller service. + * + * @return + */ + public String getType() { + return type; + } + + public void setType(String type) { + this.type = type; + } + + /** + * The state of the processor referencing a controller service. If this + * component is another service, this field is blank. + * + * @return + */ + public String getState() { + return state; + } + + public void setState(String state) { + this.state = state; + } + + /** + * The type of reference this is (Processor, ControllerService, or ReportingTask). + * @return + */ + public String getReferenceType() { + return referenceType; + } + + public void setReferenceType(String referenceType) { + this.referenceType = referenceType; + } + + /** + * The component properties. + * + * @return + */ + public Map getProperties() { + return properties; + } + + public void setProperties(Map properties) { + this.properties = properties; + } + + /** + * The descriptors for the components properties. + * + * @return + */ + public Map getDescriptors() { + return descriptors; + } + + public void setDescriptors(Map descriptors) { + this.descriptors = descriptors; + } + + /** + * Any validation error associated with this component. + * + * @return + */ + public Collection getValidationErrors() { + return validationErrors; + } + + public void setValidationErrors(Collection validationErrors) { + this.validationErrors = validationErrors; + } + + /** + * The active thread count for the referencing component. + * + * @return + */ + public Integer getActiveThreadCount() { + return activeThreadCount; + } + + public void setActiveThreadCount(Integer activeThreadCount) { + this.activeThreadCount = activeThreadCount; + } + + /** + * If this referencing component represents a ControllerService, these + * are the components that reference it. + * + * @return + */ + public Set getReferencingComponents() { + return referencingComponents; + } + + public void setReferencingComponents(Set referencingComponents) { + this.referencingComponents = referencingComponents; + } + + /** + * If this referencing component represents a ControllerService, this indicates + * whether it has already been represented in this hierarchy. + * + * @return + */ + public Boolean getReferenceCycle() { + return referenceCycle; + } + + public void setReferenceCycle(Boolean referenceCycle) { + this.referenceCycle = referenceCycle; + } + +} \ No newline at end of file diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/DocumentedTypeDTO.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/DocumentedTypeDTO.java index 5a2d78905c..7cf1b84450 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/DocumentedTypeDTO.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/DocumentedTypeDTO.java @@ -20,13 +20,13 @@ import java.util.Set; import javax.xml.bind.annotation.XmlType; /** - * Class used for providing documentation of a specified type that may be - * instantiated. + * Class used for providing documentation of a specified type. */ @XmlType(name = "documentedType") public class DocumentedTypeDTO { private String type; + private Set childTypes; private String description; private Set tags; @@ -57,7 +57,7 @@ public class DocumentedTypeDTO { } /** - * The tags associated with this type + * The tags associated with this type. * * @return */ @@ -68,4 +68,18 @@ public class DocumentedTypeDTO { public void setTags(final Set tags) { this.tags = tags; } + + /** + * Child types for this type. + * + * @return + */ + public Set getChildTypes() { + return childTypes; + } + + public void setChildTypes(Set childTypes) { + this.childTypes = childTypes; + } + } diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/FlowSnippetDTO.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/FlowSnippetDTO.java index 61c3c33762..47a6871b9d 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/FlowSnippetDTO.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/FlowSnippetDTO.java @@ -34,7 +34,8 @@ public class FlowSnippetDTO { private Set connections = new LinkedHashSet<>(); private Set labels = new LinkedHashSet<>(); private Set funnels = new LinkedHashSet<>(); - + private Set controllerServices = new LinkedHashSet<>(); + /** * The connections in this flow snippet. * @@ -138,4 +139,16 @@ public class FlowSnippetDTO { public void setRemoteProcessGroups(Set remoteProcessGroups) { this.remoteProcessGroups = remoteProcessGroups; } + + /** + * Returns the Controller Services in this flow snippet + * @return + */ + public Set getControllerServices() { + return controllerServices; + } + + public void setControllerServices(Set controllerServices) { + this.controllerServices = controllerServices; + } } diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/NiFiComponentDTO.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/NiFiComponentDTO.java index e3c844555b..2829287b66 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/NiFiComponentDTO.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/NiFiComponentDTO.java @@ -55,7 +55,7 @@ public class NiFiComponentDTO { } /** - * The id for the parent group of this component. + * The id for the parent group of this component if applicable, null otherwise. * * @return */ @@ -85,7 +85,7 @@ public class NiFiComponentDTO { } /** - * The position of this component in the UI. + * The position of this component in the UI if applicable, null otherwise. * * @return The position */ diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ProcessorConfigDTO.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ProcessorConfigDTO.java index 1481b0f9d7..63ed005026 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ProcessorConfigDTO.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ProcessorConfigDTO.java @@ -54,10 +54,10 @@ public class ProcessorConfigDTO { } /** - * The amount of time that should elapse between task executions. This will - * not affect currently scheduled tasks. + * The frequency with which to schedule the processor. The format of the value will + * depend on the value of {@link #getSchedulingStrategy()}. * - * @return The scheduling period in seconds + * @return The scheduling period */ public String getSchedulingPeriod() { return schedulingPeriod; @@ -207,7 +207,8 @@ public class ProcessorConfigDTO { } /** - * Whether of not this processor has a custom UI. + * Returns the URL for this processors custom configuration UI + * if applicable. Null otherwise. * * @return */ @@ -274,213 +275,4 @@ public class ProcessorConfigDTO { this.defaultSchedulingPeriod = defaultSchedulingPeriod; } - /** - * The allowable values for a property with a constrained set of options. - */ - @XmlType(name = "allowableValue") - public static class AllowableValueDTO { - - private String displayName; - private String value; - private String description; - - /** - * Returns the human-readable value that is allowed for this - * PropertyDescriptor - * - * @return - */ - public String getDisplayName() { - return displayName; - } - - public void setDisplayName(String displayName) { - this.displayName = displayName; - } - - /** - * Returns the value for this allowable value. - * - * @return - */ - public String getValue() { - return value; - } - - public void setValue(String value) { - this.value = value; - } - - /** - * Returns a description of this Allowable Value, or null - * if no description is given - * - * @return - */ - public String getDescription() { - return description; - } - - public void setDescription(String description) { - this.description = description; - } - - @Override - public boolean equals(final Object obj) { - if (obj == this) { - return true; - } - - if (!(obj instanceof AllowableValueDTO)) { - return false; - } - - final AllowableValueDTO other = (AllowableValueDTO) obj; - return (this.value.equals(other.getValue())); - } - - @Override - public int hashCode() { - return 23984731 + 17 * value.hashCode(); - } - } - - /** - * A description of a processor property. - */ - @XmlType(name = "propertyDescriptor") - public static class PropertyDescriptorDTO { - - private String name; - private String displayName; - private String description; - private String defaultValue; - private Set allowableValues; - private boolean required; - private boolean sensitive; - private boolean dynamic; - private boolean supportsEl; - - /** - * The set of allowable values for this property. If empty then the - * allowable values are not constrained. - * - * @return - */ - public Set getAllowableValues() { - return allowableValues; - } - - public void setAllowableValues(Set allowableValues) { - this.allowableValues = allowableValues; - } - - /** - * The default value for this property. - * - * @return - */ - public String getDefaultValue() { - return defaultValue; - } - - public void setDefaultValue(String defaultValue) { - this.defaultValue = defaultValue; - } - - /** - * And explanation of the meaning of the given property. This - * description is meant to be displayed to a user or simply provide a - * mechanism of documenting intent. - * - * @return - */ - public String getDescription() { - return description; - } - - public void setDescription(String description) { - this.description = description; - } - - /** - * The property name. - * - * @return - */ - public String getName() { - return name; - } - - public void setName(String name) { - this.name = name; - } - - /** - * The human-readable name to display to users. - * - * @return - */ - public String getDisplayName() { - return displayName; - } - - public void setDisplayName(String displayName) { - this.displayName = displayName; - } - - /** - * Determines whether the property is required for this processor. - * - * @return - */ - public boolean isRequired() { - return required; - } - - public void setRequired(boolean required) { - this.required = required; - } - - /** - * Indicates that the value for this property should be considered - * sensitive and protected whenever stored or represented. - * - * @return - */ - public boolean isSensitive() { - return sensitive; - } - - public void setSensitive(boolean sensitive) { - this.sensitive = sensitive; - } - - /** - * Indicates whether this property is dynamic. - * - * @return - */ - public boolean isDynamic() { - return dynamic; - } - - public void setDynamic(boolean dynamic) { - this.dynamic = dynamic; - } - - /** - * Specifies whether or not this property support expression language. - * - * @return - */ - public boolean getSupportsEl() { - return supportsEl; - } - - public void setSupportsEl(boolean supportsEl) { - this.supportsEl = supportsEl; - } - } - } diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/PropertyDescriptorDTO.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/PropertyDescriptorDTO.java new file mode 100644 index 0000000000..d10a324b6a --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/PropertyDescriptorDTO.java @@ -0,0 +1,243 @@ +/* + * 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.web.api.dto; + +import java.util.List; +import javax.xml.bind.annotation.XmlType; + +/** + * A description of a property. + */ +@XmlType(name = "propertyDescriptor") +public class PropertyDescriptorDTO { + + private String name; + private String displayName; + private String description; + private String defaultValue; + private List allowableValues; + private boolean required; + private boolean sensitive; + private boolean dynamic; + private boolean supportsEl; + private boolean identifiesControllerService; + + /** + * The set of allowable values for this property. If empty then the + * allowable values are not constrained. + * + * @return + */ + public List getAllowableValues() { + return allowableValues; + } + + public void setAllowableValues(List allowableValues) { + this.allowableValues = allowableValues; + } + + /** + * The default value for this property. + * + * @return + */ + public String getDefaultValue() { + return defaultValue; + } + + public void setDefaultValue(String defaultValue) { + this.defaultValue = defaultValue; + } + + /** + * And explanation of the meaning of the given property. This + * description is meant to be displayed to a user or simply provide a + * mechanism of documenting intent. + * + * @return + */ + public String getDescription() { + return description; + } + + public void setDescription(String description) { + this.description = description; + } + + /** + * The property name. + * + * @return + */ + public String getName() { + return name; + } + + public void setName(String name) { + this.name = name; + } + + /** + * The human-readable name to display to users. + * + * @return + */ + public String getDisplayName() { + return displayName; + } + + public void setDisplayName(String displayName) { + this.displayName = displayName; + } + + /** + * Determines whether the property is required for this processor. + * + * @return + */ + public boolean isRequired() { + return required; + } + + public void setRequired(boolean required) { + this.required = required; + } + + /** + * Indicates that the value for this property should be considered + * sensitive and protected whenever stored or represented. + * + * @return + */ + public boolean isSensitive() { + return sensitive; + } + + public void setSensitive(boolean sensitive) { + this.sensitive = sensitive; + } + + /** + * Indicates whether this property is dynamic. + * + * @return + */ + public boolean isDynamic() { + return dynamic; + } + + public void setDynamic(boolean dynamic) { + this.dynamic = dynamic; + } + + /** + * Specifies whether or not this property support expression language. + * + * @return + */ + public boolean getSupportsEl() { + return supportsEl; + } + + public void setSupportsEl(boolean supportsEl) { + this.supportsEl = supportsEl; + } + + /** + * Whether this descriptor represents a controller service. + * + * @return + */ + public boolean isIdentifiesControllerService() { + return identifiesControllerService; + } + + public void setIdentifiesControllerService(boolean identifiesControllerService) { + this.identifiesControllerService = identifiesControllerService; + } + + /** + * The allowable values for a property with a constrained set of options. + */ + @XmlType(name = "allowableValue") + public static class AllowableValueDTO { + + private String displayName; + private String value; + private String description; + + /** + * Returns the human-readable value that is allowed for this + * PropertyDescriptor + * + * @return + */ + public String getDisplayName() { + return displayName; + } + + public void setDisplayName(String displayName) { + this.displayName = displayName; + } + + /** + * Returns the value for this allowable value. + * + * @return + */ + public String getValue() { + return value; + } + + public void setValue(String value) { + this.value = value; + } + + /** + * Returns a description of this Allowable Value, or null + * if no description is given + * + * @return + */ + public String getDescription() { + return description; + } + + public void setDescription(String description) { + this.description = description; + } + + @Override + public boolean equals(final Object obj) { + if (obj == this) { + return true; + } + + if (!(obj instanceof AllowableValueDTO)) { + return false; + } + + final AllowableValueDTO other = (AllowableValueDTO) obj; + return (this.value.equals(other.getValue())); + } + + @Override + public int hashCode() { + return 23984731 + 17 * value.hashCode(); + } + } +} diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ReportingTaskDTO.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ReportingTaskDTO.java new file mode 100644 index 0000000000..a019f97341 --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ReportingTaskDTO.java @@ -0,0 +1,228 @@ +/* + * 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.web.api.dto; + +import java.util.Collection; +import java.util.Map; + +import javax.xml.bind.annotation.XmlType; + +/** + * Component that is capable of reporting internal NiFi state to an external service + */ +@XmlType(name = "reportingTask") +public class ReportingTaskDTO extends NiFiComponentDTO { + private String name; + private String type; + private String state; + private String availability; + private String comments; + + private String schedulingPeriod; + private String schedulingStrategy; + private Map defaultSchedulingPeriod; + + private Map properties; + private Map descriptors; + + private String customUiUrl; + private String annotationData; + + private Collection validationErrors; + private Integer activeThreadCount; + + /** + * The user-defined name of the reporting task + * @return + */ + public String getName() { + return name; + } + + public void setName(String name) { + this.name = name; + } + + /** + * The user-defined comments for the reporting task + * @return + */ + public String getComments() { + return comments; + } + + public void setComments(String comments) { + this.comments = comments; + } + + /** + * The type of reporting task + * @return + */ + public String getType() { + return type; + } + + public void setType(String type) { + this.type = type; + } + + /** + * The frequency with which to schedule the reporting task. The format of the value will + * depend on the value of {@link #getSchedulingStrategy()}. + * + * @return The scheduling period + */ + public String getSchedulingPeriod() { + return schedulingPeriod; + } + + public void setSchedulingPeriod(String schedulingPeriod) { + this.schedulingPeriod = schedulingPeriod; + } + + /** + * The current scheduling state of the reporting task + * @return + */ + public String getState() { + return state; + } + + public void setState(String state) { + this.state = state; + } + + /** + * The scheduling strategy that determines how the {@link #getSchedulingPeriod()} value should + * be interpreted + * + * @return + */ + public String getSchedulingStrategy() { + return schedulingStrategy; + } + + public void setSchedulingStrategy(String schedulingStrategy) { + this.schedulingStrategy = schedulingStrategy; + } + + /** + * Where this service is available. Possible values are CLUSTER_MANAGER_ONLY, NODE_ONLY, BOTH. + * + * @return + */ + public String getAvailability() { + return availability; + } + + public void setAvailability(String availability) { + this.availability = availability; + } + + /** + * The reporting task's properties + * @return + */ + public Map getProperties() { + return properties; + } + + public void setProperties(Map properties) { + this.properties = properties; + } + + /** + * Map of property name to descriptor + * @return + */ + public Map getDescriptors() { + return descriptors; + } + + public void setDescriptors(Map descriptors) { + this.descriptors = descriptors; + } + + /** + * Returns the URL for this reporting task custom configuration UI + * if applicable. Null otherwise. + * + * @return + */ + public String getCustomUiUrl() { + return customUiUrl; + } + + public void setCustomUiUrl(String customUiUrl) { + this.customUiUrl = customUiUrl; + } + + /** + * The currently configured annotation data for the reporting task + * @return + */ + public String getAnnotationData() { + return annotationData; + } + + public void setAnnotationData(String annotationData) { + this.annotationData = annotationData; + } + + /** + * Gets the validation errors from this reporting task. These validation errors + * represent the problems with the reporting task that must be resolved before it + * can be scheduled to run. + * + * @return The validation errors + */ + public Collection getValidationErrors() { + return validationErrors; + } + + public void setValidationErrors(Collection validationErrors) { + this.validationErrors = validationErrors; + } + + /** + * The default scheduling period for the different scheduling strategies. + * + * @return + */ + public Map getDefaultSchedulingPeriod() { + return defaultSchedulingPeriod; + } + + public void setDefaultSchedulingPeriod(Map defaultSchedulingPeriod) { + this.defaultSchedulingPeriod = defaultSchedulingPeriod; + } + + /** + * The number of active threads for this reporting task. + * + * @return + */ + public Integer getActiveThreadCount() { + return activeThreadCount; + } + + public void setActiveThreadCount(Integer activeThreadCount) { + this.activeThreadCount = activeThreadCount; + } + +} diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/RevisionDTO.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/RevisionDTO.java index e608a7efcb..3327b49815 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/RevisionDTO.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/RevisionDTO.java @@ -26,8 +26,10 @@ public class RevisionDTO { private String clientId; private Long version; + private String lastModifier; /* getters / setters */ + /** * A client identifier used to make a request. By including a client * identifier, the API can allow multiple requests without needing the @@ -60,4 +62,17 @@ public class RevisionDTO { this.version = version; } + /** + * The user that last modified the flow. + * + * @return + */ + public String getLastModifier() { + return lastModifier; + } + + public void setLastModifier(String lastModifier) { + this.lastModifier = lastModifier; + } + } diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/action/component/details/ComponentDetailsDTO.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/action/component/details/ComponentDetailsDTO.java index 58086ce285..b01a271ae1 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/action/component/details/ComponentDetailsDTO.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/action/component/details/ComponentDetailsDTO.java @@ -24,7 +24,7 @@ import javax.xml.bind.annotation.XmlType; */ @XmlType(name = "componentDetails") @XmlSeeAlso({ - ProcessorDetailsDTO.class, + ExtensionDetailsDTO.class, RemoteProcessGroupDetailsDTO.class }) public class ComponentDetailsDTO { diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/action/component/details/ProcessorDetailsDTO.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/action/component/details/ExtensionDetailsDTO.java similarity index 87% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/action/component/details/ProcessorDetailsDTO.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/action/component/details/ExtensionDetailsDTO.java index 3523f62e94..e2e49d67a2 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/action/component/details/ProcessorDetailsDTO.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/action/component/details/ExtensionDetailsDTO.java @@ -19,15 +19,15 @@ package org.apache.nifi.web.api.dto.action.component.details; import javax.xml.bind.annotation.XmlType; /** - * Processor details for an action. + * Extension details for an action. */ -@XmlType(name = "processorDetails") -public class ProcessorDetailsDTO extends ComponentDetailsDTO { +@XmlType(name = "extensionDetails") +public class ExtensionDetailsDTO extends ComponentDetailsDTO { private String type; /** - * The processors type. + * The extension type. * * @return */ diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ProcessorHistoryEntity.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ComponentHistoryEntity.java similarity index 66% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ProcessorHistoryEntity.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ComponentHistoryEntity.java index 19166f7c99..ca682110df 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ProcessorHistoryEntity.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ComponentHistoryEntity.java @@ -17,29 +17,29 @@ package org.apache.nifi.web.api.entity; import javax.xml.bind.annotation.XmlRootElement; -import org.apache.nifi.web.api.dto.ProcessorHistoryDTO; +import org.apache.nifi.web.api.dto.ComponentHistoryDTO; /** * A serialized representation of this class can be placed in the entity body of * a request or response to or from the API. This particular entity holds a - * reference to a ProcessorHistoryDTO. + * reference to a ComponentHistoryDTO. */ -@XmlRootElement(name = "processorHistoryEntity") -public class ProcessorHistoryEntity extends Entity { +@XmlRootElement(name = "componentHistoryEntity") +public class ComponentHistoryEntity extends Entity { - private ProcessorHistoryDTO propertyHistory; + private ComponentHistoryDTO componentHistory; /** - * The ProcessorHistoryDTO that is being serialized. + * The ComponentHistoryDTO that is being serialized. * - * @return The ProcessorHistoryDTO object + * @return The ComponentHistoryDTO object */ - public ProcessorHistoryDTO getProcessorHistory() { - return propertyHistory; + public ComponentHistoryDTO getComponentHistory() { + return componentHistory; } - public void setProcessorHistory(ProcessorHistoryDTO propertyHistory) { - this.propertyHistory = propertyHistory; + public void setComponentHistory(ComponentHistoryDTO componentHistory) { + this.componentHistory = componentHistory; } } diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ControllerServiceEntity.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ControllerServiceEntity.java new file mode 100644 index 0000000000..44364e712c --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ControllerServiceEntity.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.web.api.entity; + +import javax.xml.bind.annotation.XmlRootElement; +import org.apache.nifi.web.api.dto.ControllerServiceDTO; + +/** + * A serialized representation of this class can be placed in the entity body of + * a response to the API. This particular entity holds a reference to a + * controller service. + */ +@XmlRootElement(name = "controllerServiceEntity") +public class ControllerServiceEntity extends Entity { + + private ControllerServiceDTO controllerService; + + /** + * The controller service that is being serialized. + * + * @return + */ + public ControllerServiceDTO getControllerService() { + return controllerService; + } + + public void setControllerService(ControllerServiceDTO controllerService) { + this.controllerService = controllerService; + } + +} diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ControllerServiceReferencingComponentsEntity.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ControllerServiceReferencingComponentsEntity.java new file mode 100644 index 0000000000..6010f93014 --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ControllerServiceReferencingComponentsEntity.java @@ -0,0 +1,46 @@ +/* + * 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.web.api.entity; + +import java.util.Set; +import javax.xml.bind.annotation.XmlRootElement; +import org.apache.nifi.web.api.dto.ControllerServiceReferencingComponentDTO; + +/** + * A serialized representation of this class can be placed in the entity body of + * a response to the API. This particular entity holds a reference to a list of + * controller services referencing components. + */ +@XmlRootElement(name = "controllerServiceReferencingComponentsEntity") +public class ControllerServiceReferencingComponentsEntity extends Entity { + + private Set controllerServiceReferencingComponents; + + /** + * The list of controller service referencing components that are being serialized. + * + * @return + */ + public Set getControllerServiceReferencingComponents() { + return controllerServiceReferencingComponents; + } + + public void setControllerServiceReferencingComponents(Set controllerServiceReferencingComponents) { + this.controllerServiceReferencingComponents = controllerServiceReferencingComponents; + } + +} diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ControllerServiceTypesEntity.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ControllerServiceTypesEntity.java new file mode 100644 index 0000000000..dafb8c22af --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ControllerServiceTypesEntity.java @@ -0,0 +1,46 @@ +/* + * 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.web.api.entity; + +import java.util.Set; +import javax.xml.bind.annotation.XmlRootElement; +import org.apache.nifi.web.api.dto.DocumentedTypeDTO; + +/** + * A serialized representation of this class can be placed in the entity body of + * a response to the API. This particular entity holds a reference to a list of + * controller service types. + */ +@XmlRootElement(name = "controllerServiceTypesEntity") +public class ControllerServiceTypesEntity extends Entity { + + private Set controllerServiceTypes; + + /** + * The list of controller service types that are being serialized. + * + * @return + */ + public Set getControllerServiceTypes() { + return controllerServiceTypes; + } + + public void setControllerServiceTypes(Set controllerServiceTypes) { + this.controllerServiceTypes = controllerServiceTypes; + } + +} diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ControllerServicesEntity.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ControllerServicesEntity.java new file mode 100644 index 0000000000..4485b43d15 --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ControllerServicesEntity.java @@ -0,0 +1,46 @@ +/* + * 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.web.api.entity; + +import java.util.Set; +import javax.xml.bind.annotation.XmlRootElement; +import org.apache.nifi.web.api.dto.ControllerServiceDTO; + +/** + * A serialized representation of this class can be placed in the entity body of + * a response to the API. This particular entity holds a reference to a list of + * controller services. + */ +@XmlRootElement(name = "controllerServicesEntity") +public class ControllerServicesEntity extends Entity { + + private Set controllerServices; + + /** + * The list of controller services that are being serialized. + * + * @return + */ + public Set getControllerServices() { + return controllerServices; + } + + public void setControllerServices(Set controllerServices) { + this.controllerServices = controllerServices; + } + +} diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/PropertyDescriptorEntity.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/PropertyDescriptorEntity.java new file mode 100644 index 0000000000..9251b7f220 --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/PropertyDescriptorEntity.java @@ -0,0 +1,46 @@ +/* + * 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.web.api.entity; + +import javax.xml.bind.annotation.XmlRootElement; +import org.apache.nifi.web.api.dto.ProcessorDTO; +import org.apache.nifi.web.api.dto.PropertyDescriptorDTO; + +/** + * A serialized representation of this class can be placed in the entity body of + * a request or response to or from the API. This particular entity holds a + * reference to a PropertyDescriptorDTO. + */ +@XmlRootElement(name = "propertyDescriptor") +public class PropertyDescriptorEntity extends Entity { + + private PropertyDescriptorDTO propertyDescriptor; + + /** + * The PropertyDescriptorDTO that is being serialized. + * + * @return The PropertyDescriptorDTO object + */ + public PropertyDescriptorDTO getPropertyDescriptor() { + return propertyDescriptor; + } + + public void setPropertyDescriptor(PropertyDescriptorDTO propertyDescriptor) { + this.propertyDescriptor = propertyDescriptor; + } + +} diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ReportingTaskEntity.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ReportingTaskEntity.java new file mode 100644 index 0000000000..a372751712 --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ReportingTaskEntity.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.web.api.entity; + +import javax.xml.bind.annotation.XmlRootElement; +import org.apache.nifi.web.api.dto.ReportingTaskDTO; + +/** + * A serialized representation of this class can be placed in the entity body of + * a response to the API. This particular entity holds a reference to a + * reporting task. + */ +@XmlRootElement(name = "reportingTaskEntity") +public class ReportingTaskEntity extends Entity { + + private ReportingTaskDTO reportingTask; + + /** + * The reporting task that is being serialized. + * + * @return + */ + public ReportingTaskDTO getReportingTask() { + return reportingTask; + } + + public void setReportingTask(ReportingTaskDTO reportingTask) { + this.reportingTask = reportingTask; + } + +} diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ReportingTaskTypesEntity.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ReportingTaskTypesEntity.java new file mode 100644 index 0000000000..4b021efb44 --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ReportingTaskTypesEntity.java @@ -0,0 +1,46 @@ +/* + * 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.web.api.entity; + +import java.util.Set; +import javax.xml.bind.annotation.XmlRootElement; +import org.apache.nifi.web.api.dto.DocumentedTypeDTO; + +/** + * A serialized representation of this class can be placed in the entity body of + * a response to the API. This particular entity holds a reference to a list of + * reporting task types. + */ +@XmlRootElement(name = "reportingTaskTypesEntity") +public class ReportingTaskTypesEntity extends Entity { + + private Set reportingTaskTypes; + + /** + * The list of reporting task types that are being serialized. + * + * @return + */ + public Set getReportingTaskTypes() { + return reportingTaskTypes; + } + + public void setReportingTaskTypes(Set reportingTaskTypes) { + this.reportingTaskTypes = reportingTaskTypes; + } + +} diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ReportingTasksEntity.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ReportingTasksEntity.java new file mode 100644 index 0000000000..4699d5d3c4 --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ReportingTasksEntity.java @@ -0,0 +1,46 @@ +/* + * 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.web.api.entity; + +import java.util.Set; +import javax.xml.bind.annotation.XmlRootElement; +import org.apache.nifi.web.api.dto.ReportingTaskDTO; + +/** + * A serialized representation of this class can be placed in the entity body of + * a response to the API. This particular entity holds a reference to a list of + * reporting tasks. + */ +@XmlRootElement(name = "reportingTasksEntity") +public class ReportingTasksEntity extends Entity { + + private Set reportingTasks; + + /** + * The list of reporting tasks that are being serialized. + * + * @return + */ + public Set getReportingTasks() { + return reportingTasks; + } + + public void setReportingTasks(Set reportingTasks) { + this.reportingTasks = reportingTasks; + } + +} 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 fcd3ea340f..69ce8d9ca4 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 @@ -18,6 +18,7 @@ package org.apache.nifi.documentation.mock; import org.apache.nifi.controller.ControllerServiceInitializationContext; import org.apache.nifi.controller.ControllerServiceLookup; +import org.apache.nifi.logging.ComponentLog; /** * A Mock ControllerServiceInitializationContext so that ControllerServices can @@ -37,4 +38,9 @@ public class MockControllerServiceInitializationContext implements ControllerSer return new MockControllerServiceLookup(); } + @Override + public ComponentLog getLogger() { + return null; + } + } 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 f11bc6874e..5c60881cee 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 @@ -52,4 +52,14 @@ public class MockControllerServiceLookup implements ControllerServiceLookup { return Collections.emptySet(); } + @Override + public boolean isControllerServiceEnabling(String serviceIdentifier) { + return false; + } + + @Override + public String getControllerServiceName(String serviceIdentifier) { + return serviceIdentifier; + } + } 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 910ce5a5e2..dc6e236f8f 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 @@ -19,6 +19,7 @@ package org.apache.nifi.documentation.mock; import java.util.concurrent.TimeUnit; import org.apache.nifi.controller.ControllerServiceLookup; +import org.apache.nifi.logging.ComponentLog; import org.apache.nifi.reporting.ReportingInitializationContext; import org.apache.nifi.scheduling.SchedulingStrategy; @@ -26,8 +27,6 @@ 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 { @@ -60,4 +59,9 @@ public class MockReportingInitializationContext implements ReportingInitializati public SchedulingStrategy getSchedulingStrategy() { return SchedulingStrategy.TIMER_DRIVEN; } + + @Override + public ComponentLog getLogger() { + return null; + } } diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-web/pom.xml b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-web/pom.xml index f9ee703bb0..70dcc8120d 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-web/pom.xml +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-web/pom.xml @@ -32,10 +32,6 @@ org.apache.nifi nifi-properties - - org.apache.nifi - nifi-web-optimistic-locking - org.apache.nifi nifi-administration diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-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 index 012e7c7081..c8c7206947 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-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 @@ -32,12 +32,7 @@ public class ClusterContextThreadLocal { } public static ClusterContext getContext() { - ClusterContext ctx = contextHolder.get(); - if(ctx == null) { - ctx = createEmptyContext(); - contextHolder.set(ctx); - } - return ctx; + return contextHolder.get(); } public static void setContext(final ClusterContext context) { diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-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 deleted file mode 100644 index 90b8a370e2..0000000000 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-web/src/main/java/org/apache/nifi/web/ClusterAwareOptimisticLockingManager.java +++ /dev/null @@ -1,96 +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.web; - -import org.apache.nifi.cluster.context.ClusterContext; -import org.apache.nifi.cluster.context.ClusterContextThreadLocal; - -/** - * An optimistic locking manager that provides for optimistic locking in a clustered - * environment. - * - * @author unattributed - */ -public class ClusterAwareOptimisticLockingManager implements OptimisticLockingManager { - - private final OptimisticLockingManager optimisticLockingManager; - - public ClusterAwareOptimisticLockingManager(final OptimisticLockingManager optimisticLockingManager) { - this.optimisticLockingManager = optimisticLockingManager; - } - - @Override - public Revision checkRevision(Revision revision) throws InvalidRevisionException { - final Revision currentRevision = getRevision(); - if(currentRevision.equals(revision) == false) { - throw new InvalidRevisionException(String.format("Given revision %s does not match current revision %s.", revision, currentRevision)); - } else { - return revision.increment(revision.getClientId()); - } - } - - @Override - public boolean isCurrent(Revision revision) { - return getRevision().equals(revision); - } - - @Override - public Revision getRevision() { - final ClusterContext ctx = ClusterContextThreadLocal.getContext(); - if(ctx == null || ctx.getRevision() == null) { - return optimisticLockingManager.getRevision(); - } else { - return ctx.getRevision(); - } - } - - @Override - public void setRevision(final Revision revision) { - final ClusterContext ctx = ClusterContextThreadLocal.getContext(); - if(ctx != null) { - ctx.setRevision(revision); - } - optimisticLockingManager.setRevision(revision); - } - - @Override - public Revision incrementRevision() { - final Revision currentRevision = getRevision(); - final Revision incRevision = currentRevision.increment(); - setRevision(incRevision); - return incRevision; - } - - @Override - public Revision incrementRevision(final String clientId) { - final Revision currentRevision = getRevision(); - final Revision incRevision = currentRevision.increment(clientId); - setRevision(incRevision); - return incRevision; - } - - @Override - public String getLastModifier() { - return optimisticLockingManager.getLastModifier(); - } - - @Override - public void setLastModifier(final String lastModifier) { - optimisticLockingManager.setLastModifier(lastModifier); - } - -} diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/pom.xml b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/pom.xml index 7b6a4183fa..bdff00fcf0 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/pom.xml +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/pom.xml @@ -45,6 +45,10 @@ org.apache.nifi nifi-client-dto + + org.apache.nifi + nifi-web-optimistic-locking + org.apache.nifi nifi-framework-core diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-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 index eedb88fb80..c17b429518 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-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 @@ -27,14 +27,25 @@ import org.apache.nifi.cluster.protocol.StandardDataFlow; public class ClusterDataFlow { private final StandardDataFlow dataFlow; - private final NodeIdentifier primaryNodeId; + private final byte[] controllerServices; + private final byte[] reportingTasks; - public ClusterDataFlow(final StandardDataFlow dataFlow, final NodeIdentifier primaryNodeId) { + public ClusterDataFlow(final StandardDataFlow dataFlow, final NodeIdentifier primaryNodeId, final byte[] controllerServices, final byte[] reportingTasks) { this.dataFlow = dataFlow; this.primaryNodeId = primaryNodeId; + this.controllerServices = controllerServices; + this.reportingTasks = reportingTasks; } + public byte[] getControllerServices() { + return controllerServices; + } + + public byte[] getReportingTasks() { + return reportingTasks; + } + public NodeIdentifier getPrimaryNodeId() { return primaryNodeId; } diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-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 index 339d9047f3..082d65e512 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-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 @@ -17,6 +17,7 @@ package org.apache.nifi.cluster.flow; import java.util.Set; + import org.apache.nifi.cluster.protocol.NodeIdentifier; /** @@ -66,6 +67,22 @@ public interface DataFlowManagementService { */ void updatePrimaryNode(NodeIdentifier nodeId) throws DaoException; + /** + * Updates the dataflow with the given serialized form of the Controller Services that are to exist on the NCM. + * + * @param serializedControllerServices + * @throws DaoException + */ + void updateControllerServices(byte[] serializedControllerServices) throws DaoException; + + /** + * Updates the dataflow with the given serialized form of Reporting Tasks that are to exist on the NCM. + * + * @param serviceNodes + * @throws DaoException + */ + void updateReportingTasks(byte[] serializedReportingTasks) throws DaoException; + /** * Sets the state of the flow. * diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-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 index 72b594afd1..dd9d2a3c6f 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-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 @@ -111,6 +111,8 @@ public class DataFlowDaoImpl implements DataFlowDao { public static final String FLOW_XML_FILENAME = "flow.xml"; public static final String TEMPLATES_FILENAME = "templates.xml"; public static final String SNIPPETS_FILENAME = "snippets.xml"; + public static final String CONTROLLER_SERVICES_FILENAME = "controller-services.xml"; + public static final String REPORTING_TASKS_FILENAME = "reporting-tasks.xml"; public static final String CLUSTER_INFO_FILENAME = "cluster-info.xml"; private static final Logger logger = new NiFiLog(LoggerFactory.getLogger(DataFlowDaoImpl.class)); @@ -408,13 +410,7 @@ public class DataFlowDaoImpl implements DataFlowDao { final File stateFile = new File(dir, FLOW_PACKAGE); stateFile.createNewFile(); - final byte[] flowBytes = getEmptyFlowBytes(); - final byte[] templateBytes = new byte[0]; - final byte[] snippetBytes = new byte[0]; - final DataFlow dataFlow = new StandardDataFlow(flowBytes, templateBytes, snippetBytes); - - final ClusterMetadata clusterMetadata = new ClusterMetadata(); - writeDataFlow(stateFile, dataFlow, clusterMetadata); + writeDataFlow(stateFile, new ClusterDataFlow(null, null, new byte[0], new byte[0]), new ClusterMetadata()); return stateFile; } @@ -479,7 +475,9 @@ public class DataFlowDaoImpl implements DataFlowDao { byte[] templateBytes = new byte[0]; byte[] snippetBytes = new byte[0]; byte[] clusterInfoBytes = new byte[0]; - + byte[] controllerServiceBytes = new byte[0]; + byte[] reportingTaskBytes = new byte[0]; + try (final InputStream inStream = new FileInputStream(file); final TarArchiveInputStream tarIn = new TarArchiveInputStream(new BufferedInputStream(inStream))) { TarArchiveEntry tarEntry; @@ -501,6 +499,14 @@ public class DataFlowDaoImpl implements DataFlowDao { clusterInfoBytes = new byte[(int) tarEntry.getSize()]; StreamUtils.fillBuffer(tarIn, clusterInfoBytes, true); break; + case CONTROLLER_SERVICES_FILENAME: + controllerServiceBytes = new byte[(int) tarEntry.getSize()]; + StreamUtils.fillBuffer(tarIn, controllerServiceBytes, true); + break; + case REPORTING_TASKS_FILENAME: + reportingTaskBytes = new byte[(int) tarEntry.getSize()]; + StreamUtils.fillBuffer(tarIn, reportingTaskBytes, true); + break; default: throw new DaoException("Found Unexpected file in dataflow configuration: " + tarEntry.getName()); } @@ -518,7 +524,7 @@ public class DataFlowDaoImpl implements DataFlowDao { final StandardDataFlow dataFlow = new StandardDataFlow(flowBytes, templateBytes, snippetBytes); dataFlow.setAutoStartProcessors(autoStart); - return new ClusterDataFlow(dataFlow, (clusterMetadata == null) ? null : clusterMetadata.getPrimaryNodeId()); + return new ClusterDataFlow(dataFlow, (clusterMetadata == null) ? null : clusterMetadata.getPrimaryNodeId(), controllerServiceBytes, reportingTaskBytes); } private void writeDataFlow(final File file, final ClusterDataFlow clusterDataFlow) throws IOException, JAXBException { @@ -536,7 +542,7 @@ public class DataFlowDaoImpl implements DataFlowDao { clusterMetadata.setPrimaryNodeId(clusterDataFlow.getPrimaryNodeId()); // write to disk - writeDataFlow(file, dataFlow, clusterMetadata); + writeDataFlow(file, clusterDataFlow, clusterMetadata); } private void writeTarEntry(final TarArchiveOutputStream tarOut, final String filename, final byte[] bytes) throws IOException { @@ -547,14 +553,23 @@ public class DataFlowDaoImpl implements DataFlowDao { tarOut.closeArchiveEntry(); } - private void writeDataFlow(final File file, final DataFlow dataFlow, final ClusterMetadata clusterMetadata) throws IOException, JAXBException { + private void writeDataFlow(final File file, final ClusterDataFlow clusterDataFlow, final ClusterMetadata clusterMetadata) throws IOException, JAXBException { try (final OutputStream fos = new FileOutputStream(file); final TarArchiveOutputStream tarOut = new TarArchiveOutputStream(new BufferedOutputStream(fos))) { - writeTarEntry(tarOut, FLOW_XML_FILENAME, dataFlow.getFlow()); - writeTarEntry(tarOut, TEMPLATES_FILENAME, dataFlow.getTemplates()); - writeTarEntry(tarOut, SNIPPETS_FILENAME, dataFlow.getSnippets()); + final DataFlow dataFlow = clusterDataFlow.getDataFlow(); + if ( dataFlow == null ) { + writeTarEntry(tarOut, FLOW_XML_FILENAME, getEmptyFlowBytes()); + writeTarEntry(tarOut, TEMPLATES_FILENAME, new byte[0]); + writeTarEntry(tarOut, SNIPPETS_FILENAME, new byte[0]); + } else { + writeTarEntry(tarOut, FLOW_XML_FILENAME, dataFlow.getFlow()); + writeTarEntry(tarOut, TEMPLATES_FILENAME, dataFlow.getTemplates()); + writeTarEntry(tarOut, SNIPPETS_FILENAME, dataFlow.getSnippets()); + } + writeTarEntry(tarOut, CONTROLLER_SERVICES_FILENAME, clusterDataFlow.getControllerServices()); + writeTarEntry(tarOut, REPORTING_TASKS_FILENAME, clusterDataFlow.getReportingTasks()); final ByteArrayOutputStream baos = new ByteArrayOutputStream(256); writeClusterMetadata(clusterMetadata, baos); diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-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 index e135af311f..1bb8ca3c92 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-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 @@ -41,7 +41,6 @@ import org.apache.nifi.cluster.protocol.message.FlowRequestMessage; import org.apache.nifi.cluster.protocol.message.FlowResponseMessage; import org.apache.nifi.logging.NiFiLog; import org.apache.nifi.util.FormatUtils; - import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -154,17 +153,74 @@ public class DataFlowManagementServiceImpl implements DataFlowManagementService final ClusterDataFlow existingClusterDataFlow = flowDao.loadDataFlow(); final StandardDataFlow dataFlow; + final byte[] controllerServiceBytes; + final byte[] reportingTaskBytes; if (existingClusterDataFlow == null) { dataFlow = null; + controllerServiceBytes = new byte[0]; + reportingTaskBytes = new byte[0]; } else { dataFlow = existingClusterDataFlow.getDataFlow(); + controllerServiceBytes = existingClusterDataFlow.getControllerServices(); + reportingTaskBytes = existingClusterDataFlow.getReportingTasks(); } - flowDao.saveDataFlow(new ClusterDataFlow(dataFlow, nodeId)); + flowDao.saveDataFlow(new ClusterDataFlow(dataFlow, nodeId, controllerServiceBytes, reportingTaskBytes)); } finally { resourceLock.unlock("updatePrimaryNode"); } } + + + @Override + public void updateControllerServices(final byte[] controllerServiceBytes) throws DaoException { + resourceLock.lock(); + try { + final ClusterDataFlow existingClusterDataFlow = flowDao.loadDataFlow(); + + final StandardDataFlow dataFlow; + final byte[] reportingTaskBytes; + final NodeIdentifier nodeId; + if (existingClusterDataFlow == null) { + dataFlow = null; + nodeId = null; + reportingTaskBytes = new byte[0]; + } else { + dataFlow = existingClusterDataFlow.getDataFlow(); + nodeId = existingClusterDataFlow.getPrimaryNodeId(); + reportingTaskBytes = existingClusterDataFlow.getReportingTasks(); + } + + flowDao.saveDataFlow(new ClusterDataFlow(dataFlow, nodeId, controllerServiceBytes, reportingTaskBytes)); + } finally { + resourceLock.unlock("updateControllerServices"); + } + } + + @Override + public void updateReportingTasks(final byte[] reportingTaskBytes) throws DaoException { + resourceLock.lock(); + try { + final ClusterDataFlow existingClusterDataFlow = flowDao.loadDataFlow(); + + final StandardDataFlow dataFlow; + final byte[] controllerServiceBytes; + final NodeIdentifier nodeId; + if (existingClusterDataFlow == null) { + dataFlow = null; + nodeId = null; + controllerServiceBytes = null; + } else { + dataFlow = existingClusterDataFlow.getDataFlow(); + nodeId = existingClusterDataFlow.getPrimaryNodeId(); + controllerServiceBytes = existingClusterDataFlow.getControllerServices(); + } + + flowDao.saveDataFlow(new ClusterDataFlow(dataFlow, nodeId, controllerServiceBytes, reportingTaskBytes)); + } finally { + resourceLock.unlock("updateControllerServices"); + } + } @Override public PersistedFlowState getPersistedFlowState() { @@ -303,9 +359,10 @@ public class DataFlowManagementServiceImpl implements DataFlowManagementService final ClusterDataFlow existingClusterDataFlow = flowDao.loadDataFlow(); final ClusterDataFlow currentClusterDataFlow; if (existingClusterDataFlow == null) { - currentClusterDataFlow = new ClusterDataFlow(dataFlow, null); + currentClusterDataFlow = new ClusterDataFlow(dataFlow, null, new byte[0], new byte[0]); } else { - currentClusterDataFlow = new ClusterDataFlow(dataFlow, existingClusterDataFlow.getPrimaryNodeId()); + currentClusterDataFlow = new ClusterDataFlow(dataFlow, existingClusterDataFlow.getPrimaryNodeId(), + existingClusterDataFlow.getControllerServices(), existingClusterDataFlow.getReportingTasks()); } flowDao.saveDataFlow(currentClusterDataFlow); flowDao.setPersistedFlowState(PersistedFlowState.CURRENT); diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-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 index 3f966e579d..8bc73abb9d 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-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 @@ -191,6 +191,20 @@ public class NodeResponse { return clientResponse; } + /** + * If this node response has been merged returns the updated entity, + * otherwise null. Also returns null if hasThrowable() is true. The + * intent of this method is to support getting the response entity + * when it was already consumed during the merge operation. In this + * case the client response rom getClientResponse() will not support + * a getEntity(...) or getEntityInputStream() call. + * + * @return + */ + public Entity getUpdatedEntity() { + return updatedEntity; + } + /** * Creates a Response by mapping the ClientResponse values to it. Since the * ClientResponse's input stream can only be read once, this method should diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-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 index 4d5455f55b..eff523a0c4 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-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 @@ -16,12 +16,12 @@ */ package org.apache.nifi.cluster.manager.impl; -import java.io.File; +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; import java.io.IOException; import java.io.OutputStream; import java.io.Serializable; import java.net.URI; -import java.net.URL; import java.util.ArrayList; import java.util.Collection; import java.util.Collections; @@ -41,7 +41,9 @@ import java.util.TimerTask; import java.util.TreeMap; import java.util.UUID; import java.util.concurrent.CompletionService; +import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentLinkedQueue; +import java.util.concurrent.ConcurrentMap; import java.util.concurrent.ExecutorCompletionService; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; @@ -54,17 +56,20 @@ import javax.net.ssl.SSLContext; import javax.ws.rs.HttpMethod; import javax.ws.rs.WebApplicationException; import javax.ws.rs.core.StreamingOutput; -import javax.xml.XMLConstants; import javax.xml.parsers.DocumentBuilder; import javax.xml.parsers.DocumentBuilderFactory; import javax.xml.parsers.ParserConfigurationException; +import javax.xml.transform.OutputKeys; +import javax.xml.transform.Transformer; +import javax.xml.transform.TransformerException; +import javax.xml.transform.TransformerFactory; import javax.xml.transform.dom.DOMSource; -import javax.xml.validation.Schema; -import javax.xml.validation.SchemaFactory; -import javax.xml.validation.Validator; +import javax.xml.transform.stream.StreamResult; import org.apache.commons.lang3.StringUtils; import org.apache.nifi.admin.service.AuditService; +import org.apache.nifi.annotation.lifecycle.OnAdded; +import org.apache.nifi.annotation.lifecycle.OnRemoved; import org.apache.nifi.cluster.BulletinsPayload; import org.apache.nifi.cluster.HeartbeatPayload; import org.apache.nifi.cluster.context.ClusterContext; @@ -122,12 +127,18 @@ import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.controller.ControllerService; import org.apache.nifi.controller.Heartbeater; import org.apache.nifi.controller.ReportingTaskNode; +import org.apache.nifi.controller.ScheduledState; +import org.apache.nifi.controller.StandardFlowSerializer; import org.apache.nifi.controller.ValidationContextFactory; +import org.apache.nifi.controller.exception.ProcessorLifeCycleException; import org.apache.nifi.controller.reporting.ClusteredReportingTaskNode; import org.apache.nifi.controller.reporting.ReportingTaskInstantiationException; +import org.apache.nifi.controller.reporting.ReportingTaskProvider; import org.apache.nifi.controller.reporting.StandardReportingInitializationContext; +import org.apache.nifi.controller.scheduling.QuartzSchedulingAgent; import org.apache.nifi.controller.scheduling.StandardProcessScheduler; import org.apache.nifi.controller.scheduling.TimerDrivenSchedulingAgent; +import org.apache.nifi.controller.service.ControllerServiceLoader; import org.apache.nifi.controller.service.ControllerServiceNode; import org.apache.nifi.controller.service.ControllerServiceProvider; import org.apache.nifi.controller.service.StandardControllerServiceProvider; @@ -147,10 +158,12 @@ import org.apache.nifi.events.BulletinFactory; import org.apache.nifi.events.VolatileBulletinRepository; import org.apache.nifi.framework.security.util.SslContextFactory; import org.apache.nifi.io.socket.multicast.DiscoverableService; +import org.apache.nifi.logging.ComponentLog; import org.apache.nifi.logging.NiFiLog; import org.apache.nifi.nar.ExtensionManager; import org.apache.nifi.nar.NarCloseable; import org.apache.nifi.nar.NarThreadContextClassLoader; +import org.apache.nifi.processor.SimpleProcessLogger; import org.apache.nifi.processor.StandardValidationContextFactory; import org.apache.nifi.remote.RemoteResourceManager; import org.apache.nifi.remote.RemoteSiteListener; @@ -168,7 +181,11 @@ import org.apache.nifi.scheduling.SchedulingStrategy; import org.apache.nifi.util.DomUtils; import org.apache.nifi.util.FormatUtils; import org.apache.nifi.util.NiFiProperties; +import org.apache.nifi.util.ObjectHolder; +import org.apache.nifi.util.ReflectionUtils; +import org.apache.nifi.web.OptimisticLockingManager; import org.apache.nifi.web.Revision; +import org.apache.nifi.web.UpdateRevision; import org.apache.nifi.web.api.dto.FlowSnippetDTO; import org.apache.nifi.web.api.dto.NodeDTO; import org.apache.nifi.web.api.dto.ProcessGroupDTO; @@ -204,6 +221,16 @@ import org.xml.sax.SAXParseException; import com.sun.jersey.api.client.ClientResponse; +import org.apache.nifi.controller.service.ControllerServiceState; +import org.apache.nifi.web.api.dto.ControllerServiceDTO; +import org.apache.nifi.web.api.dto.ControllerServiceReferencingComponentDTO; +import org.apache.nifi.web.api.dto.ReportingTaskDTO; +import org.apache.nifi.web.api.entity.ControllerServiceEntity; +import org.apache.nifi.web.api.entity.ControllerServiceReferencingComponentsEntity; +import org.apache.nifi.web.api.entity.ControllerServicesEntity; +import org.apache.nifi.web.api.entity.ReportingTaskEntity; +import org.apache.nifi.web.api.entity.ReportingTasksEntity; + /** * Provides a cluster manager implementation. The manager federates incoming * HTTP client requests to the nodes' external API using the HTTP protocol. The @@ -222,7 +249,7 @@ import com.sun.jersey.api.client.ClientResponse; * * @author unattributed */ -public class WebClusterManager implements HttpClusterManager, ProtocolHandler, ControllerServiceProvider { +public class WebClusterManager implements HttpClusterManager, ProtocolHandler, ControllerServiceProvider, ReportingTaskProvider { public static final String ROOT_GROUP_ID_ALIAS = "root"; public static final String BULLETIN_CATEGORY = "Clustering"; @@ -279,6 +306,7 @@ public class WebClusterManager implements HttpClusterManager, ProtocolHandler, C public static final Pattern PROCESSORS_URI_PATTERN = Pattern.compile("/nifi-api/controller/process-groups/(?:(?:root)|(?:[a-f0-9\\-]{36}))/processors"); public static final Pattern PROCESSOR_URI_PATTERN = Pattern.compile("/nifi-api/controller/process-groups/(?:(?:root)|(?:[a-f0-9\\-]{36}))/processors/[a-f0-9\\-]{36}"); + public static final Pattern CLUSTER_PROCESSOR_URI_PATTERN = Pattern.compile("/nifi-api/cluster/processors/[a-f0-9\\-]{36}"); public static final Pattern REMOTE_PROCESS_GROUPS_URI_PATTERN = Pattern.compile("/nifi-api/controller/process-groups/(?:(?:root)|(?:[a-f0-9\\-]{36}))/remote-process-groups"); public static final Pattern REMOTE_PROCESS_GROUP_URI_PATTERN = Pattern.compile("/nifi-api/controller/process-groups/(?:(?:root)|(?:[a-f0-9\\-]{36}))/remote-process-groups/[a-f0-9\\-]{36}"); @@ -290,12 +318,19 @@ public class WebClusterManager implements HttpClusterManager, ProtocolHandler, C public static final String PROVENANCE_URI = "/nifi-api/controller/provenance"; public static final Pattern PROVENANCE_QUERY_URI = Pattern.compile("/nifi-api/controller/provenance/[a-f0-9\\-]{36}"); public static final Pattern PROVENANCE_EVENT_URI = Pattern.compile("/nifi-api/controller/provenance/events/[0-9]+"); - + + public static final String CONTROLLER_SERVICES_URI = "/nifi-api/controller/controller-services/node"; + public static final Pattern CONTROLLER_SERVICE_URI_PATTERN = Pattern.compile("/nifi-api/controller/controller-services/node/[a-f0-9\\-]{36}"); + public static final Pattern CONTROLLER_SERVICE_REFERENCES_URI_PATTERN = Pattern.compile("/nifi-api/controller/controller-services/node/[a-f0-9\\-]{36}/references"); + public static final String REPORTING_TASKS_URI = "/nifi-api/controller/reporting-tasks/node"; + public static final Pattern REPORTING_TASK_URI_PATTERN = Pattern.compile("/nifi-api/controller/reporting-tasks/node/[a-f0-9\\-]{36}"); + private final NiFiProperties properties; private final HttpRequestReplicator httpRequestReplicator; private final HttpResponseMapper httpResponseMapper; private final DataFlowManagementService dataFlowManagementService; private final ClusterManagerProtocolSenderListener senderListener; + private final OptimisticLockingManager optimisticLockingManager; private final StringEncryptor encryptor; private final Queue pendingHeartbeats = new ConcurrentLinkedQueue<>(); private final ReentrantReadWriteLock resourceRWLock = new ReentrantReadWriteLock(); @@ -303,12 +338,11 @@ public class WebClusterManager implements HttpClusterManager, ProtocolHandler, C private final ClusterManagerLock writeLock = new ClusterManagerLock(resourceRWLock.writeLock(), "Write"); private final Set nodes = new HashSet<>(); - private final Set reportingTasks = new HashSet<>(); + private final ConcurrentMap reportingTasks = new ConcurrentHashMap<>(); // null means the dataflow should be read from disk private StandardDataFlow cachedDataFlow = null; private NodeIdentifier primaryNodeId = null; - private Revision revision = new Revision(0L, ""); private Timer heartbeatMonitor; private Timer heartbeatProcessor; private volatile ClusterServicesBroadcaster servicesBroadcaster = null; @@ -329,7 +363,7 @@ public class WebClusterManager implements HttpClusterManager, ProtocolHandler, C public WebClusterManager(final HttpRequestReplicator httpRequestReplicator, final HttpResponseMapper httpResponseMapper, final DataFlowManagementService dataFlowManagementService, final ClusterManagerProtocolSenderListener senderListener, - final NiFiProperties properties, final StringEncryptor encryptor) { + final NiFiProperties properties, final StringEncryptor encryptor, final OptimisticLockingManager optimisticLockingManager) { if (httpRequestReplicator == null) { throw new IllegalArgumentException("HttpRequestReplicator may not be null."); @@ -348,11 +382,11 @@ public class WebClusterManager implements HttpClusterManager, ProtocolHandler, C this.httpResponseMapper = httpResponseMapper; this.dataFlowManagementService = dataFlowManagementService; this.properties = properties; - this.controllerServiceProvider = new StandardControllerServiceProvider(); this.bulletinRepository = new VolatileBulletinRepository(); this.instanceId = UUID.randomUUID().toString(); this.senderListener = senderListener; this.encryptor = encryptor; + this.optimisticLockingManager = optimisticLockingManager; senderListener.addHandler(this); senderListener.setBulletinRepository(bulletinRepository); @@ -393,9 +427,15 @@ public class WebClusterManager implements HttpClusterManager, ProtocolHandler, C public void heartbeat() { } }, this, encryptor); + + // When we construct the scheduling agents, we can pass null for a lot of the arguments because we are only + // going to be scheduling Reporting Tasks. Otherwise, it would not be okay. processScheduler.setSchedulingAgent(SchedulingStrategy.TIMER_DRIVEN, new TimerDrivenSchedulingAgent(null, reportingTaskEngine, null, encryptor)); + processScheduler.setSchedulingAgent(SchedulingStrategy.CRON_DRIVEN, new QuartzSchedulingAgent(null, reportingTaskEngine, null, encryptor)); processScheduler.setMaxThreadCount(SchedulingStrategy.TIMER_DRIVEN, 10); processScheduler.setMaxThreadCount(SchedulingStrategy.CRON_DRIVEN, 10); + + controllerServiceProvider = new StandardControllerServiceProvider(processScheduler, bulletinRepository); } public void start() throws IOException { @@ -429,14 +469,20 @@ public class WebClusterManager implements HttpClusterManager, ProtocolHandler, C } // load flow + final ClusterDataFlow clusterDataFlow; if (dataFlowManagementService.isFlowCurrent()) { - final ClusterDataFlow clusterDataFlow = dataFlowManagementService.loadDataFlow(); + clusterDataFlow = dataFlowManagementService.loadDataFlow(); cachedDataFlow = clusterDataFlow.getDataFlow(); primaryNodeId = clusterDataFlow.getPrimaryNodeId(); } else { throw new IOException("Flow is not current."); } + final byte[] serializedServices = clusterDataFlow.getControllerServices(); + if ( serializedServices != null && serializedServices.length > 0 ) { + ControllerServiceLoader.loadControllerServices(this, new ByteArrayInputStream(serializedServices), encryptor, bulletinRepository, properties.getAutoResumeState()); + } + // start multicast broadcasting service, if configured if (servicesBroadcaster != null) { servicesBroadcaster.start(); @@ -446,8 +492,10 @@ public class WebClusterManager implements HttpClusterManager, ProtocolHandler, C executeSafeModeTask(); // Load and start running Reporting Tasks - final File taskFile = new File(properties.getProperty(NiFiProperties.TASK_CONFIGURATION_FILE)); - reportingTasks.addAll(loadReportingTasks(taskFile)); + final byte[] serializedReportingTasks = clusterDataFlow.getReportingTasks(); + if ( serializedReportingTasks != null && serializedReportingTasks.length > 0 ) { + loadReportingTasks(serializedReportingTasks); + } } catch (final IOException ioe) { logger.warn("Failed to initialize cluster services due to: " + ioe, ioe); stop(); @@ -861,22 +909,17 @@ public class WebClusterManager implements HttpClusterManager, ProtocolHandler, C reconnectionThread.start(); } - private List loadReportingTasks(final File taskConfigXml) { - final List tasks = new ArrayList<>(); - if (taskConfigXml == null) { - logger.info("No controller tasks to start"); - return tasks; - } + private Map loadReportingTasks(final byte[] serialized) { + final Map tasks = new HashMap<>(); try { - final URL schemaUrl = getClass().getResource("/ReportingTaskConfiguration.xsd"); - final Document document = parse(taskConfigXml, schemaUrl); + final Document document = parse(serialized); - final NodeList tasksNodes = document.getElementsByTagName("tasks"); + final NodeList tasksNodes = document.getElementsByTagName("reportingTasks"); final Element tasksElement = (Element) tasksNodes.item(0); //optional properties for all ReportingTasks - for (final Element taskElement : DomUtils.getChildElementsByTagName(tasksElement, "task")) { + for (final Element taskElement : DomUtils.getChildElementsByTagName(tasksElement, "reportingTask")) { //add global properties common to all tasks Map properties = new HashMap<>(); @@ -901,17 +944,20 @@ public class WebClusterManager implements HttpClusterManager, ProtocolHandler, C final String taskSchedulingPeriod = DomUtils.getChild(taskElement, "schedulingPeriod").getTextContent().trim(); final String taskClass = DomUtils.getChild(taskElement, "class").getTextContent().trim(); - //optional task-specific properties - for (final Element optionalProperty : DomUtils.getChildElementsByTagName(taskElement, "property")) { - final String name = optionalProperty.getAttribute("name"); - final String value = optionalProperty.getTextContent().trim(); + final String scheduleStateValue = DomUtils.getChild(taskElement, "scheduledState").getTextContent().trim(); + final ScheduledState scheduledState = ScheduledState.valueOf(scheduleStateValue); + + // Reporting Task Properties + for (final Element property : DomUtils.getChildElementsByTagName(taskElement, "property")) { + final String name = DomUtils.getChildText(property, "name"); + final String value = DomUtils.getChildText(property, "value"); properties.put(name, value); } //set the class to be used for the configured reporting task final ReportingTaskNode reportingTaskNode; try { - reportingTaskNode = createReportingTask(taskClass, taskId); + reportingTaskNode = createReportingTask(taskClass, taskId, false); } catch (final ReportingTaskInstantiationException e) { logger.error("Unable to load reporting task {} due to {}", new Object[]{taskId, e}); if (logger.isDebugEnabled()) { @@ -922,27 +968,61 @@ public class WebClusterManager implements HttpClusterManager, ProtocolHandler, C final ReportingTask reportingTask = reportingTaskNode.getReportingTask(); - final ReportingInitializationContext config = new StandardReportingInitializationContext(taskId, taskName, schedulingStrategy, taskSchedulingPeriod, this); + final ComponentLog componentLog = new SimpleProcessLogger(taskId, reportingTask); + final ReportingInitializationContext config = new StandardReportingInitializationContext(taskId, taskName, + schedulingStrategy, taskSchedulingPeriod, componentLog, this); reportingTask.initialize(config); + final String annotationData = DomUtils.getChildText(taskElement, "annotationData"); + if ( annotationData != null ) { + reportingTaskNode.setAnnotationData(annotationData.trim()); + } + final Map resolvedProps; try (final NarCloseable narCloseable = NarCloseable.withNarLoader()) { resolvedProps = new HashMap<>(); for (final Map.Entry entry : properties.entrySet()) { final PropertyDescriptor descriptor = reportingTask.getPropertyDescriptor(entry.getKey()); - resolvedProps.put(descriptor, entry.getValue()); + if ( entry.getValue() == null ) { + resolvedProps.put(descriptor, descriptor.getDefaultValue()); + } else { + resolvedProps.put(descriptor, entry.getValue()); + } } } for (final Map.Entry entry : resolvedProps.entrySet()) { - reportingTaskNode.setProperty(entry.getKey().getName(), entry.getValue()); + if ( entry.getValue() != null ) { + reportingTaskNode.setProperty(entry.getKey().getName(), entry.getValue()); + } + } + + final String comments = DomUtils.getChildText(taskElement, "comment"); + if ( comments != null ) { + reportingTaskNode.setComments(comments); } - processScheduler.schedule(reportingTaskNode); - tasks.add(reportingTaskNode); + reportingTaskNode.setScheduledState(scheduledState); + if ( ScheduledState.RUNNING.equals(scheduledState) ) { + if ( reportingTaskNode.isValid() ) { + try { + processScheduler.schedule(reportingTaskNode); + } catch (final Exception e) { + logger.error("Failed to start {} due to {}", reportingTaskNode, e); + if ( logger.isDebugEnabled() ) { + logger.error("", e); + } + } + } else { + logger.error("Failed to start {} because it is invalid due to {}", reportingTaskNode, reportingTaskNode.getValidationErrors()); + } + } + + + tasks.put(reportingTaskNode.getIdentifier(), reportingTaskNode); } } catch (final SAXException | ParserConfigurationException | IOException | DOMException | NumberFormatException | InitializationException t) { - logger.error("Unable to load reporting tasks from {} due to {}", new Object[]{taskConfigXml, t}); + logger.error("Unable to load reporting tasks due to {}", new Object[]{t}); if (logger.isDebugEnabled()) { logger.error("", t); } @@ -951,7 +1031,9 @@ public class WebClusterManager implements HttpClusterManager, ProtocolHandler, C return tasks; } - private ReportingTaskNode createReportingTask(final String type, final String id) throws ReportingTaskInstantiationException { + + @Override + public ReportingTaskNode createReportingTask(final String type, final String id, final boolean firstTimeAdded) throws ReportingTaskInstantiationException { if (type == null) { throw new NullPointerException(); } @@ -981,14 +1063,22 @@ public class WebClusterManager implements HttpClusterManager, ProtocolHandler, C final ValidationContextFactory validationContextFactory = new StandardValidationContextFactory(this); final ReportingTaskNode taskNode = new ClusteredReportingTaskNode(task, id, processScheduler, new ClusteredEventAccess(this), bulletinRepository, controllerServiceProvider, validationContextFactory); + taskNode.setName(task.getClass().getSimpleName()); + + reportingTasks.put(id, taskNode); + if ( firstTimeAdded ) { + try (final NarCloseable x = NarCloseable.withNarLoader()) { + ReflectionUtils.invokeMethodsWithAnnotation(OnAdded.class, task); + } catch (final Exception e) { + throw new ProcessorLifeCycleException("Failed to invoke On-Added Lifecycle methods of " + task, e); + } + } + return taskNode; } - private Document parse(final File xmlFile, final URL schemaUrl) throws SAXException, ParserConfigurationException, IOException { - final SchemaFactory schemaFactory = SchemaFactory.newInstance(XMLConstants.W3C_XML_SCHEMA_NS_URI); - final Schema schema = schemaFactory.newSchema(schemaUrl); + private Document parse(final byte[] serialized) throws SAXException, ParserConfigurationException, IOException { final DocumentBuilderFactory docFactory = DocumentBuilderFactory.newInstance(); - docFactory.setSchema(schema); final DocumentBuilder builder = docFactory.newDocumentBuilder(); builder.setErrorHandler(new org.xml.sax.ErrorHandler() { @@ -1021,12 +1111,7 @@ public class WebClusterManager implements HttpClusterManager, ProtocolHandler, C }); // build the docuemnt - final Document document = builder.parse(xmlFile); - - // ensure schema compliance - final Validator validator = schema.newValidator(); - validator.validate(new DOMSource(document)); - + final Document document = builder.parse(new ByteArrayInputStream(serialized)); return document; } @@ -1287,7 +1372,19 @@ public class WebClusterManager implements HttpClusterManager, ProtocolHandler, C writeLock.unlock("handleControllerStartupFailure"); } } - + + /** + * Adds an instance of a specified controller service. + * + * @param type + * @param id + * @param properties + * @return + */ + @Override + public ControllerServiceNode createControllerService(final String type, final String id, final boolean firstTimeAdded) { + return controllerServiceProvider.createControllerService(type, id, firstTimeAdded); + } @Override public ControllerService getControllerService(String serviceIdentifier) { @@ -1310,10 +1407,15 @@ public class WebClusterManager implements HttpClusterManager, ProtocolHandler, C } @Override - public ControllerServiceNode createControllerService(final String type, final String id, final boolean firstTimeAdded) { - return controllerServiceProvider.createControllerService(type, id, firstTimeAdded); + public boolean isControllerServiceEnabling(final String serviceIdentifier) { + return controllerServiceProvider.isControllerServiceEnabling(serviceIdentifier); } + @Override + public String getControllerServiceName(final String serviceIdentifier) { + return controllerServiceProvider.getControllerServiceName(serviceIdentifier); + } + @Override public void removeControllerService(final ControllerServiceNode serviceNode) { controllerServiceProvider.removeControllerService(serviceNode); @@ -1325,11 +1427,215 @@ public class WebClusterManager implements HttpClusterManager, ProtocolHandler, C controllerServiceProvider.enableControllerService(serviceNode); } + @Override + public void enableControllerServices(final Collection serviceNodes) { + controllerServiceProvider.enableControllerServices(serviceNodes); + } + @Override public void disableControllerService(final ControllerServiceNode serviceNode) { controllerServiceProvider.disableControllerService(serviceNode); } + @Override + public Set getAllControllerServices() { + return controllerServiceProvider.getAllControllerServices(); + } + + + @Override + public void disableReferencingServices(final ControllerServiceNode serviceNode) { + controllerServiceProvider.disableReferencingServices(serviceNode); + } + + @Override + public void enableReferencingServices(final ControllerServiceNode serviceNode) { + controllerServiceProvider.enableReferencingServices(serviceNode); + } + + @Override + public void scheduleReferencingComponents(final ControllerServiceNode serviceNode) { + controllerServiceProvider.scheduleReferencingComponents(serviceNode); + } + + @Override + public void unscheduleReferencingComponents(final ControllerServiceNode serviceNode) { + controllerServiceProvider.unscheduleReferencingComponents(serviceNode); + } + + @Override + public void verifyCanEnableReferencingServices(final ControllerServiceNode serviceNode) { + controllerServiceProvider.verifyCanEnableReferencingServices(serviceNode); + } + + @Override + public void verifyCanScheduleReferencingComponents(final ControllerServiceNode serviceNode) { + controllerServiceProvider.verifyCanScheduleReferencingComponents(serviceNode); + } + + @Override + public void verifyCanDisableReferencingServices(final ControllerServiceNode serviceNode) { + controllerServiceProvider.verifyCanDisableReferencingServices(serviceNode); + } + + @Override + public void verifyCanStopReferencingComponents(final ControllerServiceNode serviceNode) { + controllerServiceProvider.verifyCanStopReferencingComponents(serviceNode); + } + + private byte[] serialize(final Document doc) throws TransformerException { + final ByteArrayOutputStream baos = new ByteArrayOutputStream(); + final DOMSource domSource = new DOMSource(doc); + final StreamResult streamResult = new StreamResult(baos); + + // configure the transformer and convert the DOM + final TransformerFactory transformFactory = TransformerFactory.newInstance(); + final Transformer transformer = transformFactory.newTransformer(); + transformer.setOutputProperty("{http://xml.apache.org/xslt}indent-amount", "2"); + transformer.setOutputProperty(OutputKeys.INDENT, "yes"); + + // transform the document to byte stream + transformer.transform(domSource, streamResult); + return baos.toByteArray(); + } + + private byte[] serializeControllerServices() throws ParserConfigurationException, TransformerException { + final DocumentBuilderFactory docFactory = DocumentBuilderFactory.newInstance(); + final DocumentBuilder docBuilder = docFactory.newDocumentBuilder(); + final Document document = docBuilder.newDocument(); + final Element rootElement = document.createElement("controllerServices"); + document.appendChild(rootElement); + + for ( final ControllerServiceNode serviceNode : getAllControllerServices() ) { + StandardFlowSerializer.addControllerService(rootElement, serviceNode, encryptor); + } + + return serialize(document); + } + + private byte[] serializeReportingTasks() throws ParserConfigurationException, TransformerException { + final DocumentBuilderFactory docFactory = DocumentBuilderFactory.newInstance(); + final DocumentBuilder docBuilder = docFactory.newDocumentBuilder(); + final Document document = docBuilder.newDocument(); + final Element rootElement = document.createElement("reportingTasks"); + document.appendChild(rootElement); + + for ( final ReportingTaskNode taskNode : getAllReportingTasks() ) { + StandardFlowSerializer.addReportingTask(rootElement, taskNode, encryptor); + } + + return serialize(document); + } + + + public void saveControllerServices() { + try { + dataFlowManagementService.updateControllerServices(serializeControllerServices()); + } catch (final Exception e) { + logger.error("Failed to save changes to NCM's Controller Services; changes may be lost on restart due to " + e); + if ( logger.isDebugEnabled() ) { + logger.error("", e); + } + + getBulletinRepository().addBulletin(BulletinFactory.createBulletin("Controller Services", Severity.ERROR.name(), + "Failed to save changes to NCM's Controller Services; changes may be lost on restart. See logs for more details.")); + } + } + + public void saveReportingTasks() { + try { + dataFlowManagementService.updateReportingTasks(serializeReportingTasks()); + } catch (final Exception e) { + logger.error("Failed to save changes to NCM's Reporting Tasks; changes may be lost on restart due to " + e); + if ( logger.isDebugEnabled() ) { + logger.error("", e); + } + + getBulletinRepository().addBulletin(BulletinFactory.createBulletin("Reporting Tasks", Severity.ERROR.name(), + "Failed to save changes to NCM's Reporting Tasks; changes may be lost on restart. See logs for more details.")); + } + } + + @Override + public Set getAllReportingTasks() { + readLock.lock(); + try { + return new HashSet<>(reportingTasks.values()); + } finally { + readLock.unlock("getReportingTasks"); + } + } + + @Override + public ReportingTaskNode getReportingTaskNode(final String taskId) { + readLock.lock(); + try { + return reportingTasks.get(taskId); + } finally { + readLock.unlock("getReportingTaskNode"); + } + } + + @Override + public void startReportingTask(final ReportingTaskNode reportingTaskNode) { + reportingTaskNode.verifyCanStart(); + processScheduler.schedule(reportingTaskNode); + } + + + @Override + public void stopReportingTask(final ReportingTaskNode reportingTaskNode) { + reportingTaskNode.verifyCanStop(); + processScheduler.unschedule(reportingTaskNode); + } + + @Override + public void removeReportingTask(final ReportingTaskNode reportingTaskNode) { + writeLock.lock(); + try { + final ReportingTaskNode existing = reportingTasks.get(reportingTaskNode.getIdentifier()); + if ( existing == null || existing != reportingTaskNode ) { + throw new IllegalStateException("Reporting Task " + reportingTaskNode + " does not exist in this Flow"); + } + + reportingTaskNode.verifyCanDelete(); + + try (final NarCloseable x = NarCloseable.withNarLoader()) { + ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnRemoved.class, reportingTaskNode.getReportingTask(), reportingTaskNode.getConfigurationContext()); + } + + for ( final Map.Entry entry : reportingTaskNode.getProperties().entrySet() ) { + final PropertyDescriptor descriptor = entry.getKey(); + if (descriptor.getControllerServiceDefinition() != null ) { + final String value = entry.getValue() == null ? descriptor.getDefaultValue() : entry.getValue(); + if ( value != null ) { + final ControllerServiceNode serviceNode = controllerServiceProvider.getControllerServiceNode(value); + if ( serviceNode != null ) { + serviceNode.removeReference(reportingTaskNode); + } + } + } + } + + reportingTasks.remove(reportingTaskNode.getIdentifier()); + } finally { + writeLock.unlock("removeReportingTask"); + } + } + + + @Override + public void disableReportingTask(final ReportingTaskNode reportingTask) { + reportingTask.verifyCanDisable(); + processScheduler.disableReportingTask(reportingTask); + } + + @Override + public void enableReportingTask(final ReportingTaskNode reportingTask) { + reportingTask.verifyCanEnable(); + processScheduler.enableReportingTask(reportingTask); + } + /** * Handle a bulletins message. @@ -1966,65 +2272,114 @@ public class WebClusterManager implements HttpClusterManager, ProtocolHandler, C // check if this request can change the flow final boolean mutableRequest = canChangeNodeState(method, uri); - // update headers to contain cluster contextual information to send to the node - final Map updatedHeaders = new HashMap<>(headers); - final ClusterContext clusterCtx = new ClusterContextImpl(); - clusterCtx.setRequestSentByClusterManager(true); // indicate request is sent from cluster manager - clusterCtx.setRevision(revision); + final ObjectHolder holder = new ObjectHolder<>(null); + final UpdateRevision federateRequest = new UpdateRevision() { + @Override + public Revision execute(Revision currentRevision) { + // update headers to contain cluster contextual information to send to the node + final Map updatedHeaders = new HashMap<>(headers); + final ClusterContext clusterCtx = new ClusterContextImpl(); + clusterCtx.setRequestSentByClusterManager(true); // indicate request is sent from cluster manager + clusterCtx.setRevision(currentRevision); - // serialize cluster context and add to request header - final String serializedClusterCtx = WebUtils.serializeObjectToHex(clusterCtx); - updatedHeaders.put(CLUSTER_CONTEXT_HTTP_HEADER, serializedClusterCtx); + // serialize cluster context and add to request header + final String serializedClusterCtx = WebUtils.serializeObjectToHex(clusterCtx); + updatedHeaders.put(CLUSTER_CONTEXT_HTTP_HEADER, serializedClusterCtx); - // if the request is mutable, we need to verify that it is a valid request for all nodes in the cluster. - if (mutableRequest) { - updatedHeaders.put(NCM_EXPECTS_HTTP_HEADER, "150-NodeContinue"); + // if the request is mutable, we need to verify that it is a valid request for all nodes in the cluster. + if (mutableRequest) { + updatedHeaders.put(NCM_EXPECTS_HTTP_HEADER, "150-NodeContinue"); - final Set nodeResponses; - if (entity == null) { - nodeResponses = httpRequestReplicator.replicate(nodeIds, method, uri, parameters, updatedHeaders); - } else { - nodeResponses = httpRequestReplicator.replicate(nodeIds, method, uri, entity, updatedHeaders); - } - - updatedHeaders.remove(NCM_EXPECTS_HTTP_HEADER); - - for (final NodeResponse response : nodeResponses) { - if (response.getStatus() != NODE_CONTINUE_STATUS_CODE) { - final String nodeDescription = response.getNodeId().getApiAddress() + ":" + response.getNodeId().getApiPort(); - final ClientResponse clientResponse = response.getClientResponse(); - if (clientResponse == null) { - throw new IllegalClusterStateException("Node " + nodeDescription + " is unable to fulfill this request due to: Unexpected Response Code " + response.getStatus()); + final Set nodeResponses; + if (entity == null) { + nodeResponses = httpRequestReplicator.replicate(nodeIds, method, uri, parameters, updatedHeaders); + } else { + nodeResponses = httpRequestReplicator.replicate(nodeIds, method, uri, entity, updatedHeaders); } - final String nodeExplanation = clientResponse.getEntity(String.class); - throw new IllegalClusterStateException("Node " + nodeDescription + " is unable to fulfill this request due to: " + nodeExplanation, response.getThrowable()); + + updatedHeaders.remove(NCM_EXPECTS_HTTP_HEADER); + + for (final NodeResponse response : nodeResponses) { + if (response.getStatus() != NODE_CONTINUE_STATUS_CODE) { + final String nodeDescription = response.getNodeId().getApiAddress() + ":" + response.getNodeId().getApiPort(); + final ClientResponse clientResponse = response.getClientResponse(); + if (clientResponse == null) { + throw new IllegalClusterStateException("Node " + nodeDescription + " is unable to fulfill this request due to: Unexpected Response Code " + response.getStatus()); + } + final String nodeExplanation = clientResponse.getEntity(String.class); + throw new IllegalClusterStateException("Node " + nodeDescription + " is unable to fulfill this request due to: " + nodeExplanation, response.getThrowable()); + } + } + + // set flow state to unknown to denote a mutable request replication in progress + logger.debug("Setting Flow State to UNKNOWN due to mutable request to {} {}", method, uri); + notifyDataFlowManagmentServiceOfFlowStateChange(PersistedFlowState.UNKNOWN); } - } - // set flow state to unknown to denote a mutable request replication in progress - logger.debug("Setting Flow State to UNKNOWN due to mutable request to {} {}", method, uri); - notifyDataFlowManagmentServiceOfFlowStateChange(PersistedFlowState.UNKNOWN); + // replicate request + final Set nodeResponses; + try { + if (entity == null) { + nodeResponses = httpRequestReplicator.replicate(nodeIds, method, uri, parameters, updatedHeaders); + } else { + nodeResponses = httpRequestReplicator.replicate(nodeIds, method, uri, entity, updatedHeaders); + } + } catch (final UriConstructionException uce) { + // request was not replicated, so mark the flow with its original state + if (mutableRequest) { + notifyDataFlowManagmentServiceOfFlowStateChange(originalPersistedFlowState); + } + + throw uce; + } + + // merge the response + final NodeResponse clientResponse = mergeResponses(uri, method, nodeResponses, mutableRequest); + holder.set(clientResponse); + + // if we have a response get the updated cluster context for auditing and revision updating + Revision updatedRevision = null; + if (mutableRequest && clientResponse != null) { + try { + // get the cluster context from the response header + final String serializedClusterContext = clientResponse.getClientResponse().getHeaders().getFirst(CLUSTER_CONTEXT_HTTP_HEADER); + if (StringUtils.isNotBlank(serializedClusterContext)) { + // deserialize object + final Serializable clusterContextObj = WebUtils.deserializeHexToObject(serializedClusterContext); + + // if we have a valid object, audit the actions + if (clusterContextObj instanceof ClusterContext) { + final ClusterContext clusterContext = (ClusterContext) clusterContextObj; + if (auditService != null) { + try { + auditService.addActions(clusterContext.getActions()); + } catch (Throwable t) { + logger.warn("Unable to record actions: " + t.getMessage()); + if (logger.isDebugEnabled()) { + logger.warn(StringUtils.EMPTY, t); + } + } + } + updatedRevision = clusterContext.getRevision(); + } + } + } catch (final ClassNotFoundException cnfe) { + logger.warn("Classpath issue detected because failed to deserialize cluster context from node response due to: " + cnfe, cnfe); + } + } + + return updatedRevision; + } + }; + + // federate the request and lock on the revision + if (mutableRequest) { + optimisticLockingManager.setRevision(federateRequest); + } else { + federateRequest.execute(optimisticLockingManager.getLastModification().getRevision()); } - - // replicate request - final Set nodeResponses; - try { - if (entity == null) { - nodeResponses = httpRequestReplicator.replicate(nodeIds, method, uri, parameters, updatedHeaders); - } else { - nodeResponses = httpRequestReplicator.replicate(nodeIds, method, uri, entity, updatedHeaders); - } - } catch (final UriConstructionException uce) { - // request was not replicated, so mark the flow with its original state - if (mutableRequest) { - notifyDataFlowManagmentServiceOfFlowStateChange(originalPersistedFlowState); - } - - throw uce; - } - - final NodeResponse clientResponse = mergeResponses(uri, method, nodeResponses, mutableRequest); - return clientResponse; + + return holder.get(); } private static boolean isProcessorsEndpoint(final URI uri, final String method) { @@ -2032,7 +2387,7 @@ public class WebClusterManager implements HttpClusterManager, ProtocolHandler, C } private static boolean isProcessorEndpoint(final URI uri, final String method) { - if (("GET".equalsIgnoreCase(method) || "PUT".equalsIgnoreCase(method)) && PROCESSOR_URI_PATTERN.matcher(uri.getPath()).matches()) { + if (("GET".equalsIgnoreCase(method) || "PUT".equalsIgnoreCase(method)) && (PROCESSOR_URI_PATTERN.matcher(uri.getPath()).matches() || CLUSTER_PROCESSOR_URI_PATTERN.matcher(uri.getPath()).matches()) ) { return true; } else if ("POST".equalsIgnoreCase(method) && PROCESSORS_URI_PATTERN.matcher(uri.getPath()).matches()) { return true; @@ -2079,13 +2434,51 @@ public class WebClusterManager implements HttpClusterManager, ProtocolHandler, C private static boolean isProvenanceEventEndpoint(final URI uri, final String method) { return "GET".equalsIgnoreCase(method) && PROVENANCE_EVENT_URI.matcher(uri.getPath()).matches(); } + + private static boolean isControllerServicesEndpoint(final URI uri, final String method) { + return "GET".equalsIgnoreCase(method) && CONTROLLER_SERVICES_URI.equals(uri.getPath()); + } + + private static boolean isControllerServiceEndpoint(final URI uri, final String method) { + if (("GET".equalsIgnoreCase(method) || "PUT".equalsIgnoreCase(method)) && CONTROLLER_SERVICE_URI_PATTERN.matcher(uri.getPath()).matches()) { + return true; + } else if ("POST".equalsIgnoreCase(method) && CONTROLLER_SERVICES_URI.equals(uri.getPath())) { + return true; + } + + return false; + } + + private static boolean isControllerServiceReferenceEndpoint(final URI uri, final String method) { + if (("GET".equalsIgnoreCase(method) || "PUT".equalsIgnoreCase(method)) && CONTROLLER_SERVICE_REFERENCES_URI_PATTERN.matcher(uri.getPath()).matches()) { + return true; + } + + return false; + } + + private static boolean isReportingTasksEndpoint(final URI uri, final String method) { + return "GET".equalsIgnoreCase(method) && REPORTING_TASKS_URI.equals(uri.getPath()); + } + + private static boolean isReportingTaskEndpoint(final URI uri, final String method) { + if (("GET".equalsIgnoreCase(method) || "PUT".equalsIgnoreCase(method)) && REPORTING_TASK_URI_PATTERN.matcher(uri.getPath()).matches()) { + return true; + } else if ("POST".equalsIgnoreCase(method) && REPORTING_TASKS_URI.equals(uri.getPath())) { + return true; + } + + return false; + } static boolean isResponseInterpreted(final URI uri, final String method) { return isProcessorsEndpoint(uri, method) || isProcessorEndpoint(uri, method) || isRemoteProcessGroupsEndpoint(uri, method) || isRemoteProcessGroupEndpoint(uri, method) || isProcessGroupEndpoint(uri, method) || isTemplateEndpoint(uri, method) || isFlowSnippetEndpoint(uri, method) - || isProvenanceQueryEndpoint(uri, method) || isProvenanceEventEndpoint(uri, method); + || isProvenanceQueryEndpoint(uri, method) || isProvenanceEventEndpoint(uri, method) + || isControllerServicesEndpoint(uri, method) || isControllerServiceEndpoint(uri, method) || isControllerServiceReferenceEndpoint(uri, method) + || isReportingTasksEndpoint(uri, method) || isReportingTaskEndpoint(uri, method); } private void mergeProcessorValidationErrors(final ProcessorDTO processor, Map processorMap) { @@ -2095,37 +2488,12 @@ public class WebClusterManager implements HttpClusterManager, ProtocolHandler, C final NodeIdentifier nodeId = nodeEntry.getKey(); final ProcessorDTO nodeProcessor = nodeEntry.getValue(); - // get the processor's validation errors and put them into a map - // where the key is the validation error and the value is the set of all - // nodes that reported that validation error. - final Collection nodeValidationErrors = nodeProcessor.getValidationErrors(); - if (nodeValidationErrors != null) { - for (final String nodeValidationError : nodeValidationErrors) { - Set nodeSet = validationErrorMap.get(nodeValidationError); - if (nodeSet == null) { - nodeSet = new HashSet<>(); - validationErrorMap.put(nodeValidationError, nodeSet); - } - nodeSet.add(nodeId); - } - } + // merge the validation errors + mergeValidationErrors(validationErrorMap, nodeId, nodeProcessor.getValidationErrors()); } - final Set normalizedValidationErrors = new HashSet<>(); - for (final Map.Entry> validationEntry : validationErrorMap.entrySet()) { - final String msg = validationEntry.getKey(); - final Set nodeIds = validationEntry.getValue(); - - if (nodeIds.size() == processorMap.size()) { - normalizedValidationErrors.add(msg); - } else { - for (final NodeIdentifier nodeId : nodeIds) { - normalizedValidationErrors.add(nodeId.getApiAddress() + ":" + nodeId.getApiPort() + " -- " + msg); - } - } - } - - processor.setValidationErrors(normalizedValidationErrors); + // set the merged the validation errors + processor.setValidationErrors(normalizedMergedValidationErrors(validationErrorMap, processorMap.size())); } private void mergeProvenanceQueryResults(final ProvenanceDTO provenanceDto, final Map resultMap, final Set problematicResponses) { @@ -2293,7 +2661,158 @@ public class WebClusterManager implements HttpClusterManager, ProtocolHandler, C remoteProcessGroup.setAuthorizationIssues(mergedAuthorizationIssues); } } + + private void mergeControllerServiceReferences(final Set referencingComponents, final Map> referencingComponentMap) { + final Map activeThreadCounts = new HashMap<>(); + final Map states = new HashMap<>(); + for (final Map.Entry> nodeEntry : referencingComponentMap.entrySet()) { + final Set nodeReferencingComponents = nodeEntry.getValue(); + // go through all the nodes referencing components + if ( nodeReferencingComponents != null ) { + for (final ControllerServiceReferencingComponentDTO nodeReferencingComponent : nodeReferencingComponents) { + // handle active thread counts + if (nodeReferencingComponent.getActiveThreadCount() != null && nodeReferencingComponent.getActiveThreadCount() > 0) { + final Integer current = activeThreadCounts.get(nodeReferencingComponent.getId()); + if (current == null) { + activeThreadCounts.put(nodeReferencingComponent.getId(), nodeReferencingComponent.getActiveThreadCount()); + } else { + activeThreadCounts.put(nodeReferencingComponent.getId(), nodeReferencingComponent.getActiveThreadCount() + current); + } + } + + // handle controller service state + final String state = states.get(nodeReferencingComponent.getId()); + if (state == null) { + if (ControllerServiceState.DISABLING.name().equals(nodeReferencingComponent.getState())) { + states.put(nodeReferencingComponent.getId(), ControllerServiceState.DISABLING.name()); + } else if (ControllerServiceState.ENABLING.name().equals(nodeReferencingComponent.getState())) { + states.put(nodeReferencingComponent.getId(), ControllerServiceState.ENABLING.name()); + } + } + } + } + } + + // go through each referencing components + for (final ControllerServiceReferencingComponentDTO referencingComponent : referencingComponents) { + final Integer activeThreadCount = activeThreadCounts.get(referencingComponent.getId()); + if (activeThreadCount != null) { + referencingComponent.setActiveThreadCount(activeThreadCount); + } + + final String state = states.get(referencingComponent.getId()); + if (state != null) { + referencingComponent.setState(state); + } + } + } + + private void mergeControllerService(final ControllerServiceDTO controllerService, final Map controllerServiceMap) { + final Map> validationErrorMap = new HashMap<>(); + final Set referencingComponents = controllerService.getReferencingComponents(); + final Map> nodeReferencingComponentsMap = new HashMap<>(); + + String state = null; + for (final Map.Entry nodeEntry : controllerServiceMap.entrySet()) { + final NodeIdentifier nodeId = nodeEntry.getKey(); + final ControllerServiceDTO nodeControllerService = nodeEntry.getValue(); + + if (state == null) { + if (ControllerServiceState.DISABLING.name().equals(nodeControllerService.getState())) { + state = ControllerServiceState.DISABLING.name(); + } else if (ControllerServiceState.ENABLING.name().equals(nodeControllerService.getState())) { + state = ControllerServiceState.ENABLING.name(); + } + } + + for (final ControllerServiceReferencingComponentDTO nodeReferencingComponents : nodeControllerService.getReferencingComponents()) { + nodeReferencingComponentsMap.put(nodeId, nodeReferencingComponents.getReferencingComponents()); + } + + // merge the validation errors + mergeValidationErrors(validationErrorMap, nodeId, nodeControllerService.getValidationErrors()); + } + + // merge the referencing components + mergeControllerServiceReferences(referencingComponents, nodeReferencingComponentsMap); + + // store the 'transition' state is applicable + if (state != null) { + controllerService.setState(state); + } + + // set the merged the validation errors + controllerService.setValidationErrors(normalizedMergedValidationErrors(validationErrorMap, controllerServiceMap.size())); + } + + private void mergeReportingTask(final ReportingTaskDTO reportingTask, final Map reportingTaskMap) { + final Map> validationErrorMap = new HashMap<>(); + + int activeThreadCount = 0; + for (final Map.Entry nodeEntry : reportingTaskMap.entrySet()) { + final NodeIdentifier nodeId = nodeEntry.getKey(); + final ReportingTaskDTO nodeReportingTask = nodeEntry.getValue(); + + if (nodeReportingTask.getActiveThreadCount() != null) { + activeThreadCount += nodeReportingTask.getActiveThreadCount(); + } + + // merge the validation errors + mergeValidationErrors(validationErrorMap, nodeId, nodeReportingTask.getValidationErrors()); + } + + // set the merged active thread counts + reportingTask.setActiveThreadCount(activeThreadCount); + + // set the merged the validation errors + reportingTask.setValidationErrors(normalizedMergedValidationErrors(validationErrorMap, reportingTaskMap.size())); + } + + /** + * Merges the validation errors into the specified map, recording the corresponding node identifier. + * + * @param validationErrorMap + * @param nodeId + * @param nodeValidationErrors + */ + public void mergeValidationErrors(final Map> validationErrorMap, final NodeIdentifier nodeId, final Collection nodeValidationErrors) { + if (nodeValidationErrors != null) { + for (final String nodeValidationError : nodeValidationErrors) { + Set nodeSet = validationErrorMap.get(nodeValidationError); + if (nodeSet == null) { + nodeSet = new HashSet<>(); + validationErrorMap.put(nodeValidationError, nodeSet); + } + nodeSet.add(nodeId); + } + } + } + + /** + * Normalizes the validation errors by prepending the corresponding nodes when the error does not exist across all nodes. + * + * @param validationErrorMap + * @param totalNodes + * @return + */ + public Set normalizedMergedValidationErrors(final Map> validationErrorMap, int totalNodes) { + final Set normalizedValidationErrors = new HashSet<>(); + for (final Map.Entry> validationEntry : validationErrorMap.entrySet()) { + final String msg = validationEntry.getKey(); + final Set nodeIds = validationEntry.getValue(); + + if (nodeIds.size() == totalNodes) { + normalizedValidationErrors.add(msg); + } else { + for (final NodeIdentifier nodeId : nodeIds) { + normalizedValidationErrors.add(nodeId.getApiAddress() + ":" + nodeId.getApiPort() + " -- " + msg); + } + } + } + return normalizedValidationErrors; + } + // requires write lock to be already acquired unless request is not mutable private NodeResponse mergeResponses(final URI uri, final String method, final Set nodeResponses, final boolean mutableRequest) { // holds the one response of all the node responses to return to the client @@ -2582,6 +3101,126 @@ public class WebClusterManager implements HttpClusterManager, ProtocolHandler, C event.setClusterNodeId(nodeId.getId()); event.setClusterNodeAddress(nodeId.getApiAddress() + ":" + nodeId.getApiPort()); + clientResponse = new NodeResponse(clientResponse, responseEntity); + } else if (hasSuccessfulClientResponse && isControllerServiceEndpoint(uri, method)) { + final ControllerServiceEntity responseEntity = clientResponse.getClientResponse().getEntity(ControllerServiceEntity.class); + final ControllerServiceDTO controllerService = responseEntity.getControllerService(); + + final Map resultsMap = new HashMap<>(); + for (final NodeResponse nodeResponse : updatedNodesMap.values()) { + if (problematicNodeResponses.contains(nodeResponse)) { + continue; + } + + final ControllerServiceEntity nodeResponseEntity = (nodeResponse == clientResponse) ? responseEntity : nodeResponse.getClientResponse().getEntity(ControllerServiceEntity.class); + final ControllerServiceDTO nodeControllerService = nodeResponseEntity.getControllerService(); + + resultsMap.put(nodeResponse.getNodeId(), nodeControllerService); + } + mergeControllerService(controllerService, resultsMap); + + clientResponse = new NodeResponse(clientResponse, responseEntity); + } else if (hasSuccessfulClientResponse && isControllerServicesEndpoint(uri, method)) { + final ControllerServicesEntity responseEntity = clientResponse.getClientResponse().getEntity(ControllerServicesEntity.class); + final Set controllerServices = responseEntity.getControllerServices(); + + final Map> controllerServiceMap = new HashMap<>(); + for (final NodeResponse nodeResponse : updatedNodesMap.values()) { + if (problematicNodeResponses.contains(nodeResponse)) { + continue; + } + + final ControllerServicesEntity nodeResponseEntity = (nodeResponse == clientResponse) ? responseEntity : nodeResponse.getClientResponse().getEntity(ControllerServicesEntity.class); + final Set nodeControllerServices = nodeResponseEntity.getControllerServices(); + + for (final ControllerServiceDTO nodeControllerService : nodeControllerServices) { + Map innerMap = controllerServiceMap.get(nodeControllerService.getId()); + if (innerMap == null) { + innerMap = new HashMap<>(); + controllerServiceMap.put(nodeControllerService.getId(), innerMap); + } + + innerMap.put(nodeResponse.getNodeId(), nodeControllerService); + } + } + + for (final ControllerServiceDTO controllerService : controllerServices) { + final String procId = controllerService.getId(); + final Map mergeMap = controllerServiceMap.get(procId); + + mergeControllerService(controllerService, mergeMap); + } + + // create a new client response + clientResponse = new NodeResponse(clientResponse, responseEntity); + } else if (hasSuccessfulClientResponse && isControllerServiceReferenceEndpoint(uri, method)) { + final ControllerServiceReferencingComponentsEntity responseEntity = clientResponse.getClientResponse().getEntity(ControllerServiceReferencingComponentsEntity.class); + final Set referencingComponents = responseEntity.getControllerServiceReferencingComponents(); + + final Map> resultsMap = new HashMap<>(); + for (final NodeResponse nodeResponse : updatedNodesMap.values()) { + if (problematicNodeResponses.contains(nodeResponse)) { + continue; + } + + final ControllerServiceReferencingComponentsEntity nodeResponseEntity = (nodeResponse == clientResponse) ? responseEntity : nodeResponse.getClientResponse().getEntity(ControllerServiceReferencingComponentsEntity.class); + final Set nodeReferencingComponents = nodeResponseEntity.getControllerServiceReferencingComponents(); + + resultsMap.put(nodeResponse.getNodeId(), nodeReferencingComponents); + } + mergeControllerServiceReferences(referencingComponents, resultsMap); + + clientResponse = new NodeResponse(clientResponse, responseEntity); + } else if (hasSuccessfulClientResponse && isReportingTaskEndpoint(uri, method)) { + final ReportingTaskEntity responseEntity = clientResponse.getClientResponse().getEntity(ReportingTaskEntity.class); + final ReportingTaskDTO reportingTask = responseEntity.getReportingTask(); + + final Map resultsMap = new HashMap<>(); + for (final NodeResponse nodeResponse : updatedNodesMap.values()) { + if (problematicNodeResponses.contains(nodeResponse)) { + continue; + } + + final ReportingTaskEntity nodeResponseEntity = (nodeResponse == clientResponse) ? responseEntity : nodeResponse.getClientResponse().getEntity(ReportingTaskEntity.class); + final ReportingTaskDTO nodeReportingTask = nodeResponseEntity.getReportingTask(); + + resultsMap.put(nodeResponse.getNodeId(), nodeReportingTask); + } + mergeReportingTask(reportingTask, resultsMap); + + clientResponse = new NodeResponse(clientResponse, responseEntity); + } else if (hasSuccessfulClientResponse && isReportingTasksEndpoint(uri, method)) { + final ReportingTasksEntity responseEntity = clientResponse.getClientResponse().getEntity(ReportingTasksEntity.class); + final Set reportingTaskSet = responseEntity.getReportingTasks(); + + final Map> reportingTaskMap = new HashMap<>(); + for (final NodeResponse nodeResponse : updatedNodesMap.values()) { + if (problematicNodeResponses.contains(nodeResponse)) { + continue; + } + + final ReportingTasksEntity nodeResponseEntity = (nodeResponse == clientResponse) ? responseEntity : nodeResponse.getClientResponse().getEntity(ReportingTasksEntity.class); + final Set nodeReportingTasks = nodeResponseEntity.getReportingTasks(); + + for (final ReportingTaskDTO nodeReportingTask : nodeReportingTasks) { + Map innerMap = reportingTaskMap.get(nodeReportingTask.getId()); + if (innerMap == null) { + innerMap = new HashMap<>(); + reportingTaskMap.put(nodeReportingTask.getId(), innerMap); + } + + innerMap.put(nodeResponse.getNodeId(), nodeReportingTask); + } + } + + for (final ReportingTaskDTO reportingTask : reportingTaskSet) { + final String procId = reportingTask.getId(); + final Map mergeMap = reportingTaskMap.get(procId); + + mergeReportingTask(reportingTask, mergeMap); + } + + // create a new client response clientResponse = new NodeResponse(clientResponse, responseEntity); } else { if (!nodeResponsesToDrain.isEmpty()) { @@ -2616,36 +3255,6 @@ public class WebClusterManager implements HttpClusterManager, ProtocolHandler, C logger.warn("All nodes failed to process URI {}. As a result, no node will be disconnected from cluster", uri); } } - - // if at least one node satisfied the request, then audit the action - if (hasClientResponse) { - try { - // get the cluster context from the response header - final String serializedClusterContext = clientResponse.getClientResponse().getHeaders().getFirst(CLUSTER_CONTEXT_HTTP_HEADER); - if (StringUtils.isNotBlank(serializedClusterContext)) { - // deserialize object - final Serializable clusterContextObj = WebUtils.deserializeHexToObject(serializedClusterContext); - - // if we have a valid object, audit the actions - if (clusterContextObj instanceof ClusterContext) { - final ClusterContext clusterContext = (ClusterContext) clusterContextObj; - if (auditService != null) { - try { - auditService.addActions(clusterContext.getActions()); - } catch (Throwable t) { - logger.warn("Unable to record actions: " + t.getMessage()); - if (logger.isDebugEnabled()) { - logger.warn(StringUtils.EMPTY, t); - } - } - } - revision = clusterContext.getRevision(); - } - } - } catch (final ClassNotFoundException cnfe) { - logger.warn("Classpath issue detected because failed to deserialize cluster context from node response due to: " + cnfe, cnfe); - } - } } return clientResponse; diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-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 index 7169730a56..d3cff3b26f 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-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 @@ -16,7 +16,6 @@ */ package org.apache.nifi.cluster.spring; -import java.nio.file.Paths; import org.apache.nifi.admin.service.AuditService; import org.apache.nifi.cluster.event.EventManager; import org.apache.nifi.cluster.firewall.ClusterNodeFirewall; @@ -26,11 +25,11 @@ import org.apache.nifi.cluster.manager.HttpResponseMapper; import org.apache.nifi.cluster.manager.impl.WebClusterManager; import org.apache.nifi.cluster.protocol.impl.ClusterManagerProtocolSenderListener; import org.apache.nifi.cluster.protocol.impl.ClusterServicesBroadcaster; -import org.apache.nifi.controller.service.ControllerServiceLoader; import org.apache.nifi.encrypt.StringEncryptor; import org.apache.nifi.io.socket.multicast.DiscoverableService; import org.apache.nifi.io.socket.multicast.DiscoverableServiceImpl; import org.apache.nifi.util.NiFiProperties; +import org.apache.nifi.web.OptimisticLockingManager; import org.springframework.beans.BeansException; import org.springframework.beans.factory.FactoryBean; import org.springframework.context.ApplicationContext; @@ -50,6 +49,8 @@ public class WebClusterManagerFactoryBean implements FactoryBean, ApplicationCon private NiFiProperties properties; private StringEncryptor encryptor; + + private OptimisticLockingManager optimisticLockingManager; @Override public Object getObject() throws Exception { @@ -62,13 +63,6 @@ public class WebClusterManagerFactoryBean implements FactoryBean, ApplicationCon */ return null; } else if (clusterManager == null) { - - // get the service configuration path (fail early) - final String serviceConfigurationFile = properties.getProperty(NiFiProperties.SERVICE_CONFIGURATION_FILE); - if (serviceConfigurationFile == null) { - throw new NullPointerException("The service configuration file has not been specified."); - } - final HttpRequestReplicator requestReplicator = applicationContext.getBean("httpRequestReplicator", HttpRequestReplicator.class); final HttpResponseMapper responseMapper = applicationContext.getBean("httpResponseMapper", HttpResponseMapper.class); final DataFlowManagementService dataFlowService = applicationContext.getBean("dataFlowManagementService", DataFlowManagementService.class); @@ -81,7 +75,8 @@ public class WebClusterManagerFactoryBean implements FactoryBean, ApplicationCon dataFlowService, senderListener, properties, - encryptor + encryptor, + optimisticLockingManager ); // set the service broadcaster @@ -106,10 +101,6 @@ public class WebClusterManagerFactoryBean implements FactoryBean, ApplicationCon // set the audit service clusterManager.setAuditService(applicationContext.getBean("auditService", AuditService.class)); - - // load the controller services - final ControllerServiceLoader serviceLoader = new ControllerServiceLoader(Paths.get(serviceConfigurationFile)); - serviceLoader.loadControllerServices(clusterManager); } return clusterManager; } @@ -136,4 +127,8 @@ public class WebClusterManagerFactoryBean implements FactoryBean, ApplicationCon public void setEncryptor(final StringEncryptor encryptor) { this.encryptor = encryptor; } + + public void setOptimisticLockingManager(OptimisticLockingManager optimisticLockingManager) { + this.optimisticLockingManager = optimisticLockingManager; + } } diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-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 index 68c29bca00..72c7bff22a 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-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 @@ -91,10 +91,14 @@ + + + + diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/.gitignore b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/.gitignore index ea8c4bf7f3..29546b567b 100755 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/.gitignore +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/.gitignore @@ -1 +1,2 @@ /target +/target/ diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/AbstractConfiguredComponent.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/AbstractConfiguredComponent.java index ef4b72aad8..c44161fac8 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/AbstractConfiguredComponent.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/AbstractConfiguredComponent.java @@ -23,6 +23,7 @@ import java.util.LinkedHashMap; import java.util.List; import java.util.Map; import java.util.Objects; +import java.util.Set; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.atomic.AtomicReference; @@ -149,6 +150,16 @@ public abstract class AbstractConfiguredComponent implements ConfigurableCompone final PropertyDescriptor descriptor = component.getPropertyDescriptor(name); String value = null; if (!descriptor.isRequired() && (value = properties.remove(descriptor)) != null) { + + if ( descriptor.getControllerServiceDefinition() != null ) { + if (value != null) { + final ControllerServiceNode oldNode = serviceProvider.getControllerServiceNode(value); + if (oldNode != null) { + oldNode.removeReference(this); + } + } + } + component.onPropertyModified(descriptor, value, null); return true; } @@ -250,12 +261,17 @@ public abstract class AbstractConfiguredComponent implements ConfigurableCompone return true; } + @Override public Collection getValidationErrors() { + return getValidationErrors(Collections.emptySet()); + } + + public Collection getValidationErrors(final Set serviceIdentifiersNotToValidate) { final List results = new ArrayList<>(); lock.lock(); try { - final ValidationContext validationContext = validationContextFactory.newValidationContext(getProperties(), getAnnotationData()); + final ValidationContext validationContext = validationContextFactory.newValidationContext(serviceIdentifiersNotToValidate, getProperties(), getAnnotationData()); final Collection validationResults; try (final NarCloseable narCloseable = NarCloseable.withNarLoader()) { diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/ProcessScheduler.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/ProcessScheduler.java index 303f5401ad..c3b6613238 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/ProcessScheduler.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/ProcessScheduler.java @@ -19,8 +19,7 @@ package org.apache.nifi.controller; import org.apache.nifi.connectable.Connectable; import org.apache.nifi.connectable.Funnel; import org.apache.nifi.connectable.Port; -import org.apache.nifi.processor.annotation.OnScheduled; -import org.apache.nifi.processor.annotation.OnUnscheduled; +import org.apache.nifi.controller.service.ControllerServiceNode; import org.apache.nifi.scheduling.SchedulingStrategy; public interface ProcessScheduler { @@ -143,4 +142,28 @@ public interface ProcessScheduler { * @param procNode */ void yield(ProcessorNode procNode); + + /** + * Stops scheduling the given Reporting Task to run + * @param taskNode + */ + void unschedule(ReportingTaskNode taskNode); + + /** + * Begins scheduling the given Reporting Task to run + * @param taskNode + */ + void schedule(ReportingTaskNode taskNode); + + /** + * Enables the Controller Service so that it can be used by Reporting Tasks and Processors + * @param service + */ + void enableControllerService(ControllerServiceNode service); + + /** + * Disables the Controller Service so that it can be updated + * @param service + */ + void disableControllerService(ControllerServiceNode service); } diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/ProcessorNode.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/ProcessorNode.java index f6786fa9a5..3189edd782 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/ProcessorNode.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/ProcessorNode.java @@ -21,6 +21,7 @@ import java.util.Set; import java.util.concurrent.TimeUnit; import org.apache.nifi.connectable.Connectable; +import org.apache.nifi.controller.service.ControllerServiceNode; import org.apache.nifi.controller.service.ControllerServiceProvider; import org.apache.nifi.logging.LogLevel; import org.apache.nifi.processor.Processor; @@ -77,4 +78,19 @@ public abstract class ProcessorNode extends AbstractConfiguredComponent implemen public abstract void setStyle(Map style); + /** + * Returns the number of threads (concurrent tasks) currently being used by this Processor + * @return + */ + public abstract int getActiveThreadCount(); + + /** + * Verifies that this Processor can be started if the provided set of + * services are enabled. This is introduced because we need to verify that all components + * can be started before starting any of them. In order to do that, we need to know that this + * component can be started if the given services are enabled, as we will then enable the given + * services before starting this component. + * @param ignoredReferences + */ + public abstract void verifyCanStart(Set ignoredReferences); } diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/ReportingTaskNode.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/ReportingTaskNode.java index fa48cb386e..c932f30058 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/ReportingTaskNode.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/ReportingTaskNode.java @@ -16,18 +16,16 @@ */ package org.apache.nifi.controller; +import java.util.Set; import java.util.concurrent.TimeUnit; +import org.apache.nifi.controller.service.ControllerServiceNode; import org.apache.nifi.reporting.ReportingContext; import org.apache.nifi.reporting.ReportingTask; import org.apache.nifi.scheduling.SchedulingStrategy; public interface ReportingTaskNode extends ConfiguredComponent { - Availability getAvailability(); - - void setAvailability(Availability availability); - void setSchedulingStrategy(SchedulingStrategy schedulingStrategy); SchedulingStrategy getSchedulingStrategy(); @@ -53,6 +51,12 @@ public interface ReportingTaskNode extends ConfiguredComponent { ConfigurationContext getConfigurationContext(); boolean isRunning(); + + /** + * Returns the number of threads (concurrent tasks) currently being used by this ReportingTask + * @return + */ + int getActiveThreadCount(); /** * Indicates the {@link ScheduledState} of this ReportingTask. A @@ -68,6 +72,20 @@ public interface ReportingTaskNode extends ConfiguredComponent { void setScheduledState(ScheduledState state); + String getComments(); + + void setComments(String comment); + + /** + * Verifies that this Reporting Task can be enabled if the provided set of + * services are enabled. This is introduced because we need to verify that all components + * can be started before starting any of them. In order to do that, we need to know that this + * component can be started if the given services are enabled, as we will then enable the given + * services before starting this component. + * @param ignoredReferences + */ + void verifyCanStart(Set ignoredReferences); + void verifyCanStart(); void verifyCanStop(); void verifyCanDisable(); diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/ValidationContextFactory.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/ValidationContextFactory.java index df3c251c85..09479d570b 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/ValidationContextFactory.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/ValidationContextFactory.java @@ -17,6 +17,7 @@ package org.apache.nifi.controller; import java.util.Map; +import java.util.Set; import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.components.ValidationContext; @@ -24,4 +25,7 @@ import org.apache.nifi.components.ValidationContext; public interface ValidationContextFactory { ValidationContext newValidationContext(Map properties, String annotationData); + + ValidationContext newValidationContext(Set serviceIdentifiersToNotValidate, Map properties, String annotationData); + } diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/exception/ControllerServiceNotFoundException.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/exception/ControllerServiceInstantiationException.java similarity index 77% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/exception/ControllerServiceNotFoundException.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/exception/ControllerServiceInstantiationException.java index 4cdbe5406f..18cfcda659 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/exception/ControllerServiceNotFoundException.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/exception/ControllerServiceInstantiationException.java @@ -16,28 +16,28 @@ */ package org.apache.nifi.controller.exception; -public class ControllerServiceNotFoundException extends RuntimeException { +public class ControllerServiceInstantiationException extends RuntimeException { private static final long serialVersionUID = -544424320587059277L; /** * Constructs a default exception */ - public ControllerServiceNotFoundException() { + public ControllerServiceInstantiationException() { super(); } /** * @param message */ - public ControllerServiceNotFoundException(String message) { + public ControllerServiceInstantiationException(String message) { super(message); } /** * @param cause */ - public ControllerServiceNotFoundException(Throwable cause) { + public ControllerServiceInstantiationException(Throwable cause) { super(cause); } @@ -45,7 +45,7 @@ public class ControllerServiceNotFoundException extends RuntimeException { * @param message * @param cause */ - public ControllerServiceNotFoundException(String message, Throwable cause) { + public ControllerServiceInstantiationException(String message, Throwable cause) { super(message, cause); } } diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/reporting/ReportingTaskProvider.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/reporting/ReportingTaskProvider.java new file mode 100644 index 0000000000..bb6f3f7086 --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/reporting/ReportingTaskProvider.java @@ -0,0 +1,103 @@ +/* + * 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.controller.reporting; + +import java.util.Set; +import org.apache.nifi.controller.ReportingTaskNode; + +/** + * A ReportingTaskProvider is responsible for providing management of, and access to, Reporting Tasks + */ +public interface ReportingTaskProvider { + + /** + * Creates a new instance of a reporting task + * + * @param type the type (fully qualified class name) of the reporting task to instantiate + * @param id the identifier for the Reporting Task + * @param firstTimeAdded whether or not this is the first time that the reporting task is being added + * to the flow. I.e., this will be true only when the user adds the reporting task to the flow, not when + * the flow is being restored after a restart of the software + * + * @return the ReportingTaskNode that is used to manage the reporting task + * + * @throws ReportingTaskInstantiationException if unable to create the Reporting Task + */ + ReportingTaskNode createReportingTask(String type, String id, boolean firstTimeAdded) throws ReportingTaskInstantiationException; + + /** + * Returns the reporting task that has the given identifier, or null if no reporting task + * exists with that ID. + * + * @param identifier + * @return + */ + ReportingTaskNode getReportingTaskNode(String identifier); + + /** + * Returns a Set of all Reporting Tasks that exist for this service provider. + * @return + */ + Set getAllReportingTasks(); + + /** + * Removes the given reporting task from the flow + * + * @param reportingTask + * + * @throws IllegalStateException if the reporting task cannot be removed because it is not stopped, or + * if the reporting task is not known in the flow + */ + void removeReportingTask(ReportingTaskNode reportingTask); + + /** + * Begins scheduling the reporting task to run and invokes appropriate lifecycle methods + * @param reportingTask + * + * @throws IllegalStateException if the ReportingTask's state is not STOPPED, or if the Reporting Task has active + * threads, or if the ReportingTask is not valid + */ + void startReportingTask(ReportingTaskNode reportingTask); + + /** + * Stops scheduling the reporting task to run and invokes appropriate lifecycle methods + * @param reportingTask + * + * @throws IllegalStateException if the ReportingTask's state is not RUNNING + */ + void stopReportingTask(ReportingTaskNode reportingTask); + + + /** + * Enables the reporting task to be scheduled to run + * @param reportingTask + * + * @throws IllegalStateException if the ReportingTask's state is not DISABLED + */ + void enableReportingTask(ReportingTaskNode reportingTask); + + + /** + * Disables the ability to schedul the reporting task to run + * + * @param reportingTask + * + * @throws IllegalStateException if the ReportingTask's state is not STOPPED, or if the Reporting Task has active + * threads + */ + void disableReportingTask(ReportingTaskNode reportingTask); +} diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/service/ControllerServiceNode.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/service/ControllerServiceNode.java index 66bad39a4a..50bf469e77 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/service/ControllerServiceNode.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/service/ControllerServiceNode.java @@ -16,7 +16,8 @@ */ package org.apache.nifi.controller.service; -import org.apache.nifi.controller.Availability; +import java.util.Set; + import org.apache.nifi.controller.ConfiguredComponent; import org.apache.nifi.controller.ControllerService; @@ -26,22 +27,42 @@ public interface ControllerServiceNode extends ConfiguredComponent { ControllerService getControllerServiceImplementation(); - Availability getAvailability(); - - void setAvailability(Availability availability); - - boolean isDisabled(); - - void setDisabled(boolean disabled); - + ControllerServiceState getState(); + void setState(ControllerServiceState state); + ControllerServiceReference getReferences(); void addReference(ConfiguredComponent referringComponent); void removeReference(ConfiguredComponent referringComponent); + void setComments(String comment); + String getComments(); + void verifyCanEnable(); void verifyCanDisable(); + + /** + * Verifies that this Controller Service can be disabled if the provided set of + * services are also disabled. This is introduced because we can have an instance + * where A references B, which references C, which references A and we want + * to disable service C. In this case, the cycle needs to not cause us to fail, + * so we want to verify that C can be disabled if A and B also are. + * + * @param ignoredReferences + */ + void verifyCanDisable(Set ignoredReferences); + + /** + * Verifies that this Controller Service can be enabled if the provided set of + * services are also enabled. This is introduced because we can have an instance where + * A reference B, which references C, which references A and we want to enable + * Service A. In this case, the cycle needs to not cause us to fail, so we want to verify + * that A can be enabled if A and B also are. + * @param ignoredReferences + */ + void verifyCanEnable(Set ignoredReferences); + void verifyCanDelete(); void verifyCanUpdate(); } diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/service/ControllerServiceProvider.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/service/ControllerServiceProvider.java index 5f2fc2ef85..1901fb6521 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/service/ControllerServiceProvider.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/service/ControllerServiceProvider.java @@ -16,6 +16,9 @@ */ package org.apache.nifi.controller.service; +import java.util.Collection; +import java.util.Set; + import org.apache.nifi.annotation.lifecycle.OnAdded; import org.apache.nifi.controller.ControllerServiceLookup; @@ -25,7 +28,7 @@ import org.apache.nifi.controller.ControllerServiceLookup; public interface ControllerServiceProvider extends ControllerServiceLookup { /** - * Creates a new Controller Service of the given type and assigns it the given id. If firstTimeadded + * Creates a new Controller Service of the specified type and assigns it the given id. If firstTimeadded * is true, calls any methods that are annotated with {@link OnAdded} * * @param type @@ -60,10 +63,88 @@ public interface ControllerServiceProvider extends ControllerServiceLookup { */ void enableControllerService(ControllerServiceNode serviceNode); + /** + * Enables the collection of services. If a service in this collection depends on another service, + * the service being depended on must either already be enabled or must be in the collection as well. + * @param serviceNodes + */ + void enableControllerServices(Collection serviceNodes); + /** * Disables the given controller service so that it cannot be used by other components. This allows * configuration to be updated or allows service to be removed. * @param serviceNode */ void disableControllerService(ControllerServiceNode serviceNode); + + /** + * Returns a Set of all Controller Services that exist for this service provider. + * @return + */ + Set getAllControllerServices(); + + /** + * Verifies that all running Processors and Reporting Tasks referencing the Controller Service (or a service + * that depends on the provided service) can be stopped. + * @param serviceNode + * + * @throws IllegalStateException if any referencing component cannot be stopped + */ + void verifyCanStopReferencingComponents(ControllerServiceNode serviceNode); + + /** + * Recursively unschedules all schedulable components (Processors and Reporting Tasks) that reference the given + * Controller Service. For any Controller services that reference this one, its schedulable referencing components will also + * be unscheduled. + * @param serviceNode + */ + void unscheduleReferencingComponents(ControllerServiceNode serviceNode); + + /** + * Verifies that all Controller Services referencing the provided Controller Service can be disabled. + * @param serviceNode + * + * @throws IllegalStateException if any referencing service cannot be disabled + */ + void verifyCanDisableReferencingServices(ControllerServiceNode serviceNode); + + /** + * Disables any Controller Service that references the provided Controller Service. This action is performed recursively + * so that if service A references B and B references C, disabling references for C will first disable A, then B. + * @param serviceNode + */ + void disableReferencingServices(ControllerServiceNode serviceNode); + + /** + * Verifies that all Controller Services referencing the provided ControllerService can be enabled. + * @param serviceNode + * + * @throws IllegalStateException if any referencing component cannot be enabled + */ + void verifyCanEnableReferencingServices(ControllerServiceNode serviceNode); + + + /** + * Enables all Controller Services that are referencing the given service. If Service A references Service B and Service + * B references serviceNode, Service A and B will both be enabled. + * @param serviceNode + */ + void enableReferencingServices(ControllerServiceNode serviceNode); + + /** + * Verifies that all enabled Processors referencing the ControllerService (or a service that depends on + * the provided service) can be scheduled to run. + * @param serviceNode + * + * @throws IllegalStateException if any referencing component cannot be scheduled + */ + void verifyCanScheduleReferencingComponents(ControllerServiceNode serviceNode); + + /** + * Schedules any schedulable component (Processor, ReportingTask) that is referencing the given Controller Service + * to run. This is performed recursively, so if a Processor is referencing Service A, which is referencing serviceNode, + * then the Processor will also be started. + * @param serviceNode + */ + void scheduleReferencingComponents(ControllerServiceNode serviceNode); } diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/service/ControllerServiceReference.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/service/ControllerServiceReference.java index 5cb676f494..67ffb6cb44 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/service/ControllerServiceReference.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/service/ControllerServiceReference.java @@ -41,10 +41,11 @@ public interface ControllerServiceReference { Set getReferencingComponents(); /** - * Returns a {@link Set} of all Processors and Reporting Tasks that are - * referencing the Controller Service and are running, in addition to all + * Returns a {@link Set} of all Processors, Reporting Tasks, and Controller Services that are + * referencing the Controller Service and are running (in the case of Processors and Reporting Tasks) + * or enabled (in the case of Controller Services) * * @return */ - Set getRunningReferences(); + Set getActiveReferences(); } diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/service/ControllerServiceState.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/service/ControllerServiceState.java new file mode 100644 index 0000000000..2ed8fd9316 --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/service/ControllerServiceState.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.controller.service; + + +/** + * Represents the valid states for a Controller Service. + */ +public enum ControllerServiceState { + /** + * Controller Service is disabled and cannot be used. + */ + DISABLED, + + /** + * Controller Service has been disabled but has not yet finished its lifecycle + * methods. + */ + DISABLING, + + /** + * Controller Service has been enabled but has not yet finished its lifecycle methods. + */ + ENABLING, + + /** + * Controller Service has been enabled and has finished its lifecycle methods. The Controller SErvice + * is ready to be used. + */ + ENABLED; +} diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java index 06ef203644..f3fb67c464 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java @@ -52,6 +52,7 @@ import org.apache.commons.lang3.StringUtils; import org.apache.nifi.admin.service.UserService; import org.apache.nifi.annotation.lifecycle.OnAdded; import org.apache.nifi.annotation.lifecycle.OnRemoved; +import org.apache.nifi.annotation.lifecycle.OnShutdown; import org.apache.nifi.cluster.BulletinsPayload; import org.apache.nifi.cluster.HeartbeatPayload; import org.apache.nifi.cluster.protocol.DataFlow; @@ -62,6 +63,7 @@ import org.apache.nifi.cluster.protocol.NodeProtocolSender; import org.apache.nifi.cluster.protocol.UnknownServiceAddressException; import org.apache.nifi.cluster.protocol.message.HeartbeatMessage; import org.apache.nifi.cluster.protocol.message.NodeBulletinsMessage; +import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.connectable.Connectable; import org.apache.nifi.connectable.ConnectableType; import org.apache.nifi.connectable.Connection; @@ -77,6 +79,8 @@ import org.apache.nifi.controller.exception.ProcessorLifeCycleException; import org.apache.nifi.controller.label.Label; import org.apache.nifi.controller.label.StandardLabel; import org.apache.nifi.controller.reporting.ReportingTaskInstantiationException; +import org.apache.nifi.controller.reporting.ReportingTaskProvider; +import org.apache.nifi.controller.reporting.StandardReportingInitializationContext; import org.apache.nifi.controller.reporting.StandardReportingTaskNode; import org.apache.nifi.controller.repository.ContentRepository; import org.apache.nifi.controller.repository.CounterRepository; @@ -103,6 +107,7 @@ import org.apache.nifi.controller.scheduling.StandardProcessScheduler; import org.apache.nifi.controller.scheduling.TimerDrivenSchedulingAgent; import org.apache.nifi.controller.service.ControllerServiceNode; import org.apache.nifi.controller.service.ControllerServiceProvider; +import org.apache.nifi.controller.service.StandardConfigurationContext; import org.apache.nifi.controller.service.StandardControllerServiceProvider; import org.apache.nifi.controller.status.ConnectionStatus; import org.apache.nifi.controller.status.PortStatus; @@ -129,6 +134,7 @@ import org.apache.nifi.groups.ProcessGroup; import org.apache.nifi.groups.RemoteProcessGroup; import org.apache.nifi.groups.RemoteProcessGroupPortDescriptor; import org.apache.nifi.groups.StandardProcessGroup; +import org.apache.nifi.logging.ComponentLog; import org.apache.nifi.logging.LogLevel; import org.apache.nifi.logging.LogRepository; import org.apache.nifi.logging.LogRepositoryFactory; @@ -161,6 +167,8 @@ import org.apache.nifi.remote.protocol.socket.SocketFlowFileServerProtocol; import org.apache.nifi.reporting.Bulletin; import org.apache.nifi.reporting.BulletinRepository; import org.apache.nifi.reporting.EventAccess; +import org.apache.nifi.reporting.InitializationException; +import org.apache.nifi.reporting.ReportingInitializationContext; import org.apache.nifi.reporting.ReportingTask; import org.apache.nifi.reporting.Severity; import org.apache.nifi.scheduling.SchedulingStrategy; @@ -170,6 +178,7 @@ import org.apache.nifi.util.NiFiProperties; import org.apache.nifi.util.ReflectionUtils; import org.apache.nifi.web.api.dto.ConnectableDTO; import org.apache.nifi.web.api.dto.ConnectionDTO; +import org.apache.nifi.web.api.dto.ControllerServiceDTO; import org.apache.nifi.web.api.dto.FlowSnippetDTO; import org.apache.nifi.web.api.dto.FunnelDTO; import org.apache.nifi.web.api.dto.LabelDTO; @@ -189,7 +198,7 @@ import org.slf4j.LoggerFactory; import com.sun.jersey.api.client.ClientHandlerException; -public class FlowController implements EventAccess, ControllerServiceProvider, Heartbeater, QueueProvider { +public class FlowController implements EventAccess, ControllerServiceProvider, ReportingTaskProvider, Heartbeater, QueueProvider { // default repository implementations public static final String DEFAULT_FLOWFILE_REPO_IMPLEMENTATION = "org.apache.nifi.controller.repository.WriteAheadFlowFileRepository"; @@ -374,7 +383,6 @@ public class FlowController implements EventAccess, ControllerServiceProvider, H this.properties = properties; sslContext = SslContextFactory.createSslContext(properties, false); extensionManager = new ExtensionManager(); - controllerServiceProvider = new StandardControllerServiceProvider(); timerDrivenEngineRef = new AtomicReference<>(new FlowEngine(maxTimerDrivenThreads.get(), "Timer-Driven Process")); eventDrivenEngineRef = new AtomicReference<>(new FlowEngine(maxEventDrivenThreads.get(), "Event-Driven Process")); @@ -398,6 +406,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, H processScheduler = new StandardProcessScheduler(this, this, encryptor); eventDrivenWorkerQueue = new EventDrivenWorkerQueue(false, false, processScheduler); + controllerServiceProvider = new StandardControllerServiceProvider(processScheduler, bulletinRepository); final ProcessContextFactory contextFactory = new ProcessContextFactory(contentRepository, flowFileRepository, flowFileEventRepository, counterRepositoryRef.get(), provenanceEventRepository); processScheduler.setSchedulingAgent(SchedulingStrategy.EVENT_DRIVEN, new EventDrivenSchedulingAgent( @@ -593,7 +602,10 @@ public class FlowController implements EventAccess, ControllerServiceProvider, H startConnectable(connectable); } } catch (final Throwable t) { - LOG.error("Unable to start {} due to {}", new Object[]{connectable, t}); + LOG.error("Unable to start {} due to {}", new Object[]{connectable, t.toString()}); + if ( LOG.isDebugEnabled() ) { + LOG.error("", t); + } } } @@ -1063,7 +1075,23 @@ public class FlowController implements EventAccess, ControllerServiceProvider, H // Trigger any processors' methods marked with @OnShutdown to be called rootGroup.shutdown(); - + + // invoke any methods annotated with @OnShutdown on Controller Services + for ( final ControllerServiceNode serviceNode : getAllControllerServices() ) { + try (final NarCloseable narCloseable = NarCloseable.withNarLoader()) { + final ConfigurationContext configContext = new StandardConfigurationContext(serviceNode, controllerServiceProvider); + ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnShutdown.class, serviceNode.getControllerServiceImplementation(), configContext); + } + } + + // invoke any methods annotated with @OnShutdown on Reporting Tasks + for ( final ReportingTaskNode taskNode : getAllReportingTasks() ) { + final ConfigurationContext configContext = taskNode.getConfigurationContext(); + try (final NarCloseable narCloseable = NarCloseable.withNarLoader()) { + ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnShutdown.class, taskNode.getReportingTask(), configContext); + } + } + try { this.timerDrivenEngineRef.get().awaitTermination(gracefulShutdownSeconds / 2, TimeUnit.SECONDS); this.eventDrivenEngineRef.get().awaitTermination(gracefulShutdownSeconds / 2, TimeUnit.SECONDS); @@ -1401,6 +1429,30 @@ public class FlowController implements EventAccess, ControllerServiceProvider, H try { validateSnippetContents(requireNonNull(group), dto); + // + // Instantiate Controller Services + // + for ( final ControllerServiceDTO controllerServiceDTO : dto.getControllerServices() ) { + final ControllerServiceNode serviceNode = createControllerService(controllerServiceDTO.getType(), controllerServiceDTO.getId(), true); + + serviceNode.setAnnotationData(controllerServiceDTO.getAnnotationData()); + serviceNode.setComments(controllerServiceDTO.getComments()); + serviceNode.setName(controllerServiceDTO.getName()); + } + + // configure controller services. We do this after creating all of them in case 1 service + // references another service. + for ( final ControllerServiceDTO controllerServiceDTO : dto.getControllerServices() ) { + final String serviceId = controllerServiceDTO.getId(); + final ControllerServiceNode serviceNode = getControllerServiceNode(serviceId); + + for ( final Map.Entry entry : controllerServiceDTO.getProperties().entrySet() ) { + if ( entry.getValue() != null ) { + serviceNode.setProperty(entry.getKey(), entry.getValue()); + } + } + } + // // Instantiate the labels // @@ -1411,7 +1463,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, H label.setSize(new Size(labelDTO.getWidth(), labelDTO.getHeight())); } - // TODO: Update the label's "style" + label.setStyle(labelDTO.getStyle()); group.addLabel(label); } @@ -1737,14 +1789,18 @@ public class FlowController implements EventAccess, ControllerServiceProvider, H } // validate that all Processor Types and Prioritizer Types are valid - final List processorClasses = new ArrayList<>(); + final Set processorClasses = new HashSet<>(); for (final Class c : ExtensionManager.getExtensions(Processor.class)) { processorClasses.add(c.getName()); } - final List prioritizerClasses = new ArrayList<>(); + final Set prioritizerClasses = new HashSet<>(); for (final Class c : ExtensionManager.getExtensions(FlowFilePrioritizer.class)) { prioritizerClasses.add(c.getName()); } + final Set controllerServiceClasses = new HashSet<>(); + for (final Class c : ExtensionManager.getExtensions(ControllerService.class)) { + controllerServiceClasses.add(c.getName()); + } final Set allProcs = new HashSet<>(); final Set allConns = new HashSet<>(); @@ -1760,6 +1816,15 @@ public class FlowController implements EventAccess, ControllerServiceProvider, H throw new IllegalStateException("Invalid Processor Type: " + proc.getType()); } } + + final Set controllerServices = templateContents.getControllerServices(); + if (controllerServices != null) { + for (final ControllerServiceDTO service : controllerServices) { + if (!controllerServiceClasses.contains(service.getType())) { + throw new IllegalStateException("Invalid Controller Service Type: " + service.getType()); + } + } + } for (final ConnectionDTO conn : allConns) { final List prioritizers = conn.getPrioritizers(); @@ -2480,17 +2545,20 @@ public class FlowController implements EventAccess, ControllerServiceProvider, H lookupGroup(groupId).stopProcessing(); } - public ReportingTaskNode createReportingTask(final String type, String id) throws ReportingTaskInstantiationException { - return createReportingTask(type, id, true); + public ReportingTaskNode createReportingTask(final String type) throws ReportingTaskInstantiationException { + return createReportingTask(type, true); } - public ReportingTaskNode createReportingTask(final String type, String id, final boolean firstTimeAdded) throws ReportingTaskInstantiationException { - if (type == null) { + public ReportingTaskNode createReportingTask(final String type, final boolean firstTimeAdded) throws ReportingTaskInstantiationException { + return createReportingTask(type, UUID.randomUUID().toString(), firstTimeAdded); + } + + @Override + public ReportingTaskNode createReportingTask(final String type, final String id, final boolean firstTimeAdded) throws ReportingTaskInstantiationException { + if (type == null || id == null) { throw new NullPointerException(); } - - id = requireNonNull(id).intern(); - + ReportingTask task = null; final ClassLoader ctxClassLoader = Thread.currentThread().getContextClassLoader(); try { @@ -2516,8 +2584,19 @@ public class FlowController implements EventAccess, ControllerServiceProvider, H final ValidationContextFactory validationContextFactory = new StandardValidationContextFactory(controllerServiceProvider); final ReportingTaskNode taskNode = new StandardReportingTaskNode(task, id, this, processScheduler, validationContextFactory); + taskNode.setName(task.getClass().getSimpleName()); if ( firstTimeAdded ) { + final ComponentLog componentLog = new SimpleProcessLogger(id, taskNode.getReportingTask()); + final ReportingInitializationContext config = new StandardReportingInitializationContext(id, taskNode.getName(), + SchedulingStrategy.TIMER_DRIVEN, "1 min", componentLog, this); + + try { + task.initialize(config); + } catch (final InitializationException ie) { + throw new ReportingTaskInstantiationException("Failed to initialize reporting task of type " + type, ie); + } + try (final NarCloseable x = NarCloseable.withNarLoader()) { ReflectionUtils.invokeMethodsWithAnnotation(OnAdded.class, task); } catch (final Exception e) { @@ -2529,30 +2608,33 @@ public class FlowController implements EventAccess, ControllerServiceProvider, H return taskNode; } + @Override public ReportingTaskNode getReportingTaskNode(final String taskId) { return reportingTasks.get(taskId); } + @Override public void startReportingTask(final ReportingTaskNode reportingTaskNode) { if (isTerminated()) { throw new IllegalStateException("Cannot start reporting task " + reportingTaskNode + " because the controller is terminated"); } reportingTaskNode.verifyCanStart(); - - processScheduler.schedule(reportingTaskNode); + processScheduler.schedule(reportingTaskNode); } + + @Override public void stopReportingTask(final ReportingTaskNode reportingTaskNode) { if (isTerminated()) { return; } reportingTaskNode.verifyCanStop(); - processScheduler.unschedule(reportingTaskNode); } + @Override public void removeReportingTask(final ReportingTaskNode reportingTaskNode) { final ReportingTaskNode existing = reportingTasks.get(reportingTaskNode.getIdentifier()); if ( existing == null || existing != reportingTaskNode ) { @@ -2565,43 +2647,100 @@ public class FlowController implements EventAccess, ControllerServiceProvider, H ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnRemoved.class, reportingTaskNode.getReportingTask(), reportingTaskNode.getConfigurationContext()); } + for ( final Map.Entry entry : reportingTaskNode.getProperties().entrySet() ) { + final PropertyDescriptor descriptor = entry.getKey(); + if (descriptor.getControllerServiceDefinition() != null ) { + final String value = entry.getValue() == null ? descriptor.getDefaultValue() : entry.getValue(); + if ( value != null ) { + final ControllerServiceNode serviceNode = controllerServiceProvider.getControllerServiceNode(value); + if ( serviceNode != null ) { + serviceNode.removeReference(reportingTaskNode); + } + } + } + } + reportingTasks.remove(reportingTaskNode.getIdentifier()); } - Collection getReportingTasks() { - return reportingTasks.values(); + @Override + public Set getAllReportingTasks() { + return new HashSet<>(reportingTasks.values()); } - + @Override + public ControllerServiceNode createControllerService(final String type, final String id, final boolean firstTimeAdded) { + return controllerServiceProvider.createControllerService(type, id, firstTimeAdded); + } + + @Override public void enableReportingTask(final ReportingTaskNode reportingTaskNode) { reportingTaskNode.verifyCanEnable(); - processScheduler.enableReportingTask(reportingTaskNode); } + @Override public void disableReportingTask(final ReportingTaskNode reportingTaskNode) { reportingTaskNode.verifyCanDisable(); - processScheduler.disableReportingTask(reportingTaskNode); } + @Override + public void disableReferencingServices(final ControllerServiceNode serviceNode) { + controllerServiceProvider.disableReferencingServices(serviceNode); + } + + @Override + public void enableReferencingServices(final ControllerServiceNode serviceNode) { + controllerServiceProvider.enableReferencingServices(serviceNode); + } + + @Override + public void scheduleReferencingComponents(final ControllerServiceNode serviceNode) { + controllerServiceProvider.scheduleReferencingComponents(serviceNode); + } + + @Override + public void unscheduleReferencingComponents(final ControllerServiceNode serviceNode) { + controllerServiceProvider.unscheduleReferencingComponents(serviceNode); + } + @Override public void enableControllerService(final ControllerServiceNode serviceNode) { - serviceNode.verifyCanEnable(); controllerServiceProvider.enableControllerService(serviceNode); } + @Override + public void enableControllerServices(final Collection serviceNodes) { + controllerServiceProvider.enableControllerServices(serviceNodes); + } + @Override public void disableControllerService(final ControllerServiceNode serviceNode) { serviceNode.verifyCanDisable(); controllerServiceProvider.disableControllerService(serviceNode); } - + @Override - public ControllerServiceNode createControllerService(final String type, final String id, final boolean firstTimeAdded) { - return controllerServiceProvider.createControllerService(type, id.intern(), firstTimeAdded); + public void verifyCanEnableReferencingServices(final ControllerServiceNode serviceNode) { + controllerServiceProvider.verifyCanEnableReferencingServices(serviceNode); + } + + @Override + public void verifyCanScheduleReferencingComponents(final ControllerServiceNode serviceNode) { + controllerServiceProvider.verifyCanScheduleReferencingComponents(serviceNode); } + @Override + public void verifyCanDisableReferencingServices(final ControllerServiceNode serviceNode) { + controllerServiceProvider.verifyCanDisableReferencingServices(serviceNode); + } + + @Override + public void verifyCanStopReferencingComponents(final ControllerServiceNode serviceNode) { + controllerServiceProvider.verifyCanStopReferencingComponents(serviceNode); + } + @Override public ControllerService getControllerService(final String serviceIdentifier) { return controllerServiceProvider.getControllerService(serviceIdentifier); @@ -2623,10 +2762,24 @@ public class FlowController implements EventAccess, ControllerServiceProvider, H } @Override + public boolean isControllerServiceEnabling(final String serviceIdentifier) { + return controllerServiceProvider.isControllerServiceEnabling(serviceIdentifier); + } + + @Override + public String getControllerServiceName(final String serviceIdentifier) { + return controllerServiceProvider.getControllerServiceName(serviceIdentifier); + } + public void removeControllerService(final ControllerServiceNode serviceNode) { controllerServiceProvider.removeControllerService(serviceNode); } + @Override + public Set getAllControllerServices() { + return controllerServiceProvider.getAllControllerServices(); + } + // // Counters // diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowFromDOMFactory.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowFromDOMFactory.java index c67181a063..85ad1590ee 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowFromDOMFactory.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowFromDOMFactory.java @@ -26,6 +26,7 @@ import java.util.Set; import java.util.concurrent.TimeUnit; import org.apache.nifi.connectable.Size; +import org.apache.nifi.controller.service.ControllerServiceState; import org.apache.nifi.encrypt.StringEncryptor; import org.apache.nifi.groups.RemoteProcessGroupPortDescriptor; import org.apache.nifi.remote.StandardRemoteProcessGroupPortDescriptor; @@ -33,6 +34,7 @@ import org.apache.nifi.scheduling.SchedulingStrategy; import org.apache.nifi.util.DomUtils; import org.apache.nifi.web.api.dto.ConnectableDTO; import org.apache.nifi.web.api.dto.ConnectionDTO; +import org.apache.nifi.web.api.dto.ControllerServiceDTO; import org.apache.nifi.web.api.dto.FlowSnippetDTO; import org.apache.nifi.web.api.dto.FunnelDTO; import org.apache.nifi.web.api.dto.LabelDTO; @@ -42,7 +44,7 @@ import org.apache.nifi.web.api.dto.ProcessGroupDTO; import org.apache.nifi.web.api.dto.ProcessorConfigDTO; import org.apache.nifi.web.api.dto.ProcessorDTO; import org.apache.nifi.web.api.dto.RemoteProcessGroupDTO; - +import org.apache.nifi.web.api.dto.ReportingTaskDTO; import org.w3c.dom.Element; import org.w3c.dom.NodeList; @@ -77,6 +79,40 @@ public class FlowFromDOMFactory { return styles; } + + public static ControllerServiceDTO getControllerService(final Element element, final StringEncryptor encryptor) { + final ControllerServiceDTO dto = new ControllerServiceDTO(); + + dto.setId(getString(element, "id")); + dto.setName(getString(element, "name")); + dto.setComments(getString(element, "comment")); + dto.setType(getString(element, "class")); + + final boolean enabled = getBoolean(element, "enabled"); + dto.setState(enabled ? ControllerServiceState.ENABLED.name() : ControllerServiceState.DISABLED.name()); + + dto.setProperties(getProperties(element, encryptor)); + dto.setAnnotationData(getString(element, "annotationData")); + + return dto; + } + + public static ReportingTaskDTO getReportingTask(final Element element, final StringEncryptor encryptor) { + final ReportingTaskDTO dto = new ReportingTaskDTO(); + + dto.setId(getString(element, "id")); + dto.setName(getString(element, "name")); + dto.setComments(getString(element, "comment")); + dto.setType(getString(element, "class")); + dto.setSchedulingPeriod(getString(element, "schedulingPeriod")); + dto.setState(getString(element, "scheduledState")); + dto.setSchedulingStrategy(getString(element, "schedulingStrategy")); + + dto.setProperties(getProperties(element, encryptor)); + dto.setAnnotationData(getString(element, "annotationData")); + + return dto; + } public static ProcessGroupDTO getProcessGroup(final String parentId, final Element element, final StringEncryptor encryptor) { final ProcessGroupDTO dto = new ProcessGroupDTO(); @@ -310,7 +346,6 @@ public class FlowFromDOMFactory { final ProcessorConfigDTO configDto = new ProcessorConfigDTO(); dto.setConfig(configDto); configDto.setComments(getString(element, "comment")); - configDto.setAnnotationData(getString(element, "annotationData")); configDto.setConcurrentlySchedulableTaskCount(getInt(element, "maxConcurrentTasks")); final String schedulingPeriod = getString(element, "schedulingPeriod"); configDto.setSchedulingPeriod(schedulingPeriod); @@ -334,14 +369,8 @@ public class FlowFromDOMFactory { configDto.setRunDurationMillis(TimeUnit.NANOSECONDS.toMillis(runDurationNanos)); } - final LinkedHashMap properties = new LinkedHashMap<>(); - final List propertyNodeList = getChildrenByTagName(element, "property"); - for (final Element propertyElement : propertyNodeList) { - final String name = getString(propertyElement, "name"); - final String value = decrypt(getString(propertyElement, "value"), encryptor); - properties.put(name, value); - } - configDto.setProperties(properties); + configDto.setProperties(getProperties(element, encryptor)); + configDto.setAnnotationData(getString(element, "annotationData")); final Set autoTerminatedRelationships = new HashSet<>(); final List autoTerminateList = getChildrenByTagName(element, "autoTerminatedRelationship"); @@ -353,6 +382,17 @@ public class FlowFromDOMFactory { return dto; } + private static LinkedHashMap getProperties(final Element element, final StringEncryptor encryptor) { + final LinkedHashMap properties = new LinkedHashMap<>(); + final List propertyNodeList = getChildrenByTagName(element, "property"); + for (final Element propertyElement : propertyNodeList) { + final String name = getString(propertyElement, "name"); + final String value = decrypt(getString(propertyElement, "value"), encryptor); + properties.put(name, value); + } + return properties; + } + private static String getString(final Element element, final String childElementName) { final List nodeList = getChildrenByTagName(element, childElementName); if (nodeList == null || nodeList.isEmpty()) { diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardFlowSerializer.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardFlowSerializer.java index e08a94dffb..7cd9d3bb06 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardFlowSerializer.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardFlowSerializer.java @@ -40,6 +40,8 @@ import org.apache.nifi.connectable.Port; import org.apache.nifi.connectable.Position; import org.apache.nifi.connectable.Size; import org.apache.nifi.controller.label.Label; +import org.apache.nifi.controller.service.ControllerServiceNode; +import org.apache.nifi.controller.service.ControllerServiceState; import org.apache.nifi.encrypt.StringEncryptor; import org.apache.nifi.flowfile.FlowFilePrioritizer; import org.apache.nifi.groups.ProcessGroup; @@ -47,7 +49,6 @@ import org.apache.nifi.groups.RemoteProcessGroup; import org.apache.nifi.processor.Relationship; import org.apache.nifi.remote.RemoteGroupPort; import org.apache.nifi.remote.RootGroupPort; - import org.w3c.dom.DOMException; import org.w3c.dom.Document; import org.w3c.dom.Element; @@ -79,6 +80,18 @@ public class StandardFlowSerializer implements FlowSerializer { addTextElement(rootNode, "maxTimerDrivenThreadCount", controller.getMaxTimerDrivenThreadCount()); addTextElement(rootNode, "maxEventDrivenThreadCount", controller.getMaxEventDrivenThreadCount()); addProcessGroup(rootNode, controller.getGroup(controller.getRootGroupId()), "rootGroup"); + + final Element controllerServicesNode = doc.createElement("controllerServices"); + rootNode.appendChild(controllerServicesNode); + for ( final ControllerServiceNode serviceNode : controller.getAllControllerServices() ) { + addControllerService(controllerServicesNode, serviceNode, encryptor); + } + + final Element reportingTasksNode = doc.createElement("reportingTasks"); + rootNode.appendChild(reportingTasksNode); + for ( final ReportingTaskNode taskNode : controller.getAllReportingTasks() ) { + addReportingTask(reportingTasksNode, taskNode, encryptor); + } final DOMSource domSource = new DOMSource(doc); final StreamResult streamResult = new StreamResult(new BufferedOutputStream(os)); @@ -300,8 +313,16 @@ public class StandardFlowSerializer implements FlowSerializer { addTextElement(element, "schedulingStrategy", processor.getSchedulingStrategy().name()); addTextElement(element, "runDurationNanos", processor.getRunDuration(TimeUnit.NANOSECONDS)); - // properties. - for (final Map.Entry entry : processor.getProperties().entrySet()) { + addConfiguration(element, processor.getProperties(), processor.getAnnotationData(), encryptor); + + for (final Relationship rel : processor.getAutoTerminatedRelationships()) { + addTextElement(element, "autoTerminatedRelationship", rel.getName()); + } + } + + private static void addConfiguration(final Element element, final Map properties, final String annotationData, final StringEncryptor encryptor) { + final Document doc = element.getOwnerDocument(); + for (final Map.Entry entry : properties.entrySet()) { final PropertyDescriptor descriptor = entry.getKey(); String value = entry.getValue(); @@ -322,14 +343,9 @@ public class StandardFlowSerializer implements FlowSerializer { element.appendChild(propElement); } - final String annotationData = processor.getAnnotationData(); if (annotationData != null) { addTextElement(element, "annotationData", annotationData); } - - for (final Relationship rel : processor.getAutoTerminatedRelationships()) { - addTextElement(element, "autoTerminatedRelationship", rel.getName()); - } } private void addConnection(final Element parentElement, final Connection connection) { @@ -390,11 +406,43 @@ public class StandardFlowSerializer implements FlowSerializer { parentElement.appendChild(element); } - private void addTextElement(final Element element, final String name, final long value) { + + public static void addControllerService(final Element element, final ControllerServiceNode serviceNode, final StringEncryptor encryptor) { + final Element serviceElement = element.getOwnerDocument().createElement("controllerService"); + addTextElement(serviceElement, "id", serviceNode.getIdentifier()); + addTextElement(serviceElement, "name", serviceNode.getName()); + addTextElement(serviceElement, "comment", serviceNode.getComments()); + addTextElement(serviceElement, "class", serviceNode.getControllerServiceImplementation().getClass().getCanonicalName()); + + final ControllerServiceState state = serviceNode.getState(); + final boolean enabled = (state == ControllerServiceState.ENABLED || state == ControllerServiceState.ENABLING); + addTextElement(serviceElement, "enabled", String.valueOf(enabled)); + + addConfiguration(serviceElement, serviceNode.getProperties(), serviceNode.getAnnotationData(), encryptor); + + element.appendChild(serviceElement); + } + + public static void addReportingTask(final Element element, final ReportingTaskNode taskNode, final StringEncryptor encryptor) { + final Element taskElement = element.getOwnerDocument().createElement("reportingTask"); + addTextElement(taskElement, "id", taskNode.getIdentifier()); + addTextElement(taskElement, "name", taskNode.getName()); + addTextElement(taskElement, "comment", taskNode.getComments()); + addTextElement(taskElement, "class", taskNode.getReportingTask().getClass().getCanonicalName()); + addTextElement(taskElement, "schedulingPeriod", taskNode.getSchedulingPeriod()); + addTextElement(taskElement, "scheduledState", taskNode.getScheduledState().name()); + addTextElement(taskElement, "schedulingStrategy", taskNode.getSchedulingStrategy().name()); + + addConfiguration(taskElement, taskNode.getProperties(), taskNode.getAnnotationData(), encryptor); + + element.appendChild(taskElement); + } + + private static void addTextElement(final Element element, final String name, final long value) { addTextElement(element, name, String.valueOf(value)); } - private void addTextElement(final Element element, final String name, final String value) { + private static void addTextElement(final Element element, final String name, final String value) { final Document doc = element.getOwnerDocument(); final Element toAdd = doc.createElement(name); toAdd.setTextContent(value); diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardFlowService.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardFlowService.java index 64ce5c4142..fcfee83e7c 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardFlowService.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardFlowService.java @@ -81,8 +81,6 @@ public class StandardFlowService implements FlowService, ProtocolHandler { private final FlowController controller; private final Path flowXml; - private final Path taskConfigXml; - private final Path serviceConfigXml; private final FlowConfigurationDAO dao; private final int gracefulShutdownSeconds; private final boolean autoResumeState; @@ -154,14 +152,12 @@ public class StandardFlowService implements FlowService, ProtocolHandler { this.controller = controller; this.encryptor = encryptor; flowXml = Paths.get(properties.getProperty(NiFiProperties.FLOW_CONFIGURATION_FILE)); - taskConfigXml = Paths.get(properties.getProperty(NiFiProperties.TASK_CONFIGURATION_FILE)); - serviceConfigXml = Paths.get(properties.getProperty(NiFiProperties.SERVICE_CONFIGURATION_FILE)); gracefulShutdownSeconds = (int) FormatUtils.getTimeDuration(properties.getProperty(NiFiProperties.FLOW_CONTROLLER_GRACEFUL_SHUTDOWN_PERIOD), TimeUnit.SECONDS); autoResumeState = properties.getAutoResumeState(); connectionRetryMillis = (int) FormatUtils.getTimeDuration(properties.getClusterManagerFlowRetrievalDelay(), TimeUnit.MILLISECONDS); - dao = new StandardXMLFlowConfigurationDAO(flowXml, taskConfigXml, serviceConfigXml, encryptor); + dao = new StandardXMLFlowConfigurationDAO(flowXml, encryptor); if (configuredForClustering) { @@ -605,7 +601,6 @@ public class StandardFlowService implements FlowService, ProtocolHandler { if (firstControllerInitialization) { // load the controller services logger.debug("Loading controller services"); - dao.loadControllerServices(controller); } // load the flow @@ -622,7 +617,7 @@ public class StandardFlowService implements FlowService, ProtocolHandler { logger.debug("First controller initialization. Loading reporting tasks and initializing controller."); // load the controller tasks - dao.loadReportingTasks(controller); +// dao.loadReportingTasks(controller); // initialize the flow controller.initializeFlow(); diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardFlowSynchronizer.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardFlowSynchronizer.java index 05a8f019d8..201482c015 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardFlowSynchronizer.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardFlowSynchronizer.java @@ -26,6 +26,7 @@ import java.nio.file.Path; import java.nio.file.StandardOpenOption; import java.security.NoSuchAlgorithmException; import java.util.ArrayList; +import java.util.Collections; import java.util.HashSet; import java.util.List; import java.util.Map; @@ -40,6 +41,7 @@ import javax.xml.parsers.ParserConfigurationException; import javax.xml.validation.Schema; import javax.xml.validation.SchemaFactory; +import org.apache.commons.collections4.CollectionUtils; import org.apache.nifi.cluster.protocol.DataFlow; import org.apache.nifi.cluster.protocol.StandardDataFlow; import org.apache.nifi.connectable.Connectable; @@ -51,23 +53,35 @@ import org.apache.nifi.connectable.Position; import org.apache.nifi.connectable.Size; import org.apache.nifi.controller.exception.ProcessorInstantiationException; import org.apache.nifi.controller.label.Label; +import org.apache.nifi.controller.reporting.ReportingTaskInstantiationException; +import org.apache.nifi.controller.reporting.StandardReportingInitializationContext; +import org.apache.nifi.controller.service.ControllerServiceLoader; +import org.apache.nifi.controller.service.ControllerServiceNode; +import org.apache.nifi.controller.service.ControllerServiceState; +import org.apache.nifi.encrypt.StringEncryptor; import org.apache.nifi.events.BulletinFactory; -import org.apache.nifi.util.file.FileUtils; import org.apache.nifi.fingerprint.FingerprintException; import org.apache.nifi.fingerprint.FingerprintFactory; import org.apache.nifi.flowfile.FlowFilePrioritizer; import org.apache.nifi.groups.ProcessGroup; import org.apache.nifi.groups.RemoteProcessGroup; import org.apache.nifi.groups.RemoteProcessGroupPortDescriptor; +import org.apache.nifi.logging.ComponentLog; import org.apache.nifi.logging.LogLevel; import org.apache.nifi.processor.Relationship; +import org.apache.nifi.processor.SimpleProcessLogger; import org.apache.nifi.remote.RemoteGroupPort; import org.apache.nifi.remote.RootGroupPort; +import org.apache.nifi.reporting.InitializationException; +import org.apache.nifi.reporting.ReportingInitializationContext; import org.apache.nifi.reporting.Severity; import org.apache.nifi.scheduling.SchedulingStrategy; +import org.apache.nifi.util.DomUtils; import org.apache.nifi.util.NiFiProperties; +import org.apache.nifi.util.file.FileUtils; import org.apache.nifi.web.api.dto.ConnectableDTO; import org.apache.nifi.web.api.dto.ConnectionDTO; +import org.apache.nifi.web.api.dto.ControllerServiceDTO; import org.apache.nifi.web.api.dto.FlowSnippetDTO; import org.apache.nifi.web.api.dto.FunnelDTO; import org.apache.nifi.web.api.dto.LabelDTO; @@ -77,9 +91,7 @@ import org.apache.nifi.web.api.dto.ProcessGroupDTO; import org.apache.nifi.web.api.dto.ProcessorConfigDTO; import org.apache.nifi.web.api.dto.ProcessorDTO; import org.apache.nifi.web.api.dto.RemoteProcessGroupDTO; - -import org.apache.commons.collections4.CollectionUtils; -import org.apache.nifi.encrypt.StringEncryptor; +import org.apache.nifi.web.api.dto.ReportingTaskDTO; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.w3c.dom.Document; @@ -96,9 +108,11 @@ public class StandardFlowSynchronizer implements FlowSynchronizer { private static final Logger logger = LoggerFactory.getLogger(StandardFlowSynchronizer.class); public static final URL FLOW_XSD_RESOURCE = StandardFlowSynchronizer.class.getResource("/FlowConfiguration.xsd"); private final StringEncryptor encryptor; + private final boolean autoResumeState; public StandardFlowSynchronizer(final StringEncryptor encryptor) { this.encryptor = encryptor; + autoResumeState = NiFiProperties.getInstance().getAutoResumeState(); } public static boolean isEmpty(final DataFlow dataFlow, final StringEncryptor encryptor) { @@ -157,10 +171,26 @@ public class StandardFlowSynchronizer implements FlowSynchronizer { controller.setMaxEventDrivenThreadCount(maxThreadCount / 3); } + final Element reportingTasksElement = (Element) DomUtils.getChild(rootElement, "reportingTasks"); + final List taskElements; + if ( reportingTasksElement == null ) { + taskElements = Collections.emptyList(); + } else { + taskElements = DomUtils.getChildElementsByTagName(reportingTasksElement, "reportingTask"); + } + + final Element controllerServicesElement = (Element) DomUtils.getChild(rootElement, "controllerServices"); + final List controllerServiceElements; + if ( controllerServicesElement == null ) { + controllerServiceElements = Collections.emptyList(); + } else { + controllerServiceElements = DomUtils.getChildElementsByTagName(controllerServicesElement, "controllerService"); + } + logger.trace("Parsing process group from DOM"); final Element rootGroupElement = (Element) rootElement.getElementsByTagName("rootGroup").item(0); final ProcessGroupDTO rootGroupDto = FlowFromDOMFactory.getProcessGroup(null, rootGroupElement, encryptor); - existingFlowEmpty = isEmpty(rootGroupDto); + existingFlowEmpty = taskElements.isEmpty() && controllerServiceElements.isEmpty() && isEmpty(rootGroupDto); logger.debug("Existing Flow Empty = {}", existingFlowEmpty); } } @@ -200,37 +230,64 @@ public class StandardFlowSynchronizer implements FlowSynchronizer { // create document by parsing proposed flow bytes logger.trace("Parsing proposed flow bytes as DOM document"); final Document configuration = parseFlowBytes(proposedFlow.getFlow()); - + // attempt to sync controller with proposed flow try { if (configuration != null) { - // get the root element - final Element rootElement = (Element) configuration.getElementsByTagName("flowController").item(0); - - // set controller config - logger.trace("Updating flow config"); - final Integer maxThreadCount = getInteger(rootElement, "maxThreadCount"); - if (maxThreadCount == null) { - controller.setMaxTimerDrivenThreadCount(getInt(rootElement, "maxTimerDrivenThreadCount")); - controller.setMaxEventDrivenThreadCount(getInt(rootElement, "maxEventDrivenThreadCount")); - } else { - controller.setMaxTimerDrivenThreadCount(maxThreadCount * 2 / 3); - controller.setMaxEventDrivenThreadCount(maxThreadCount / 3); - } - - // get the root group XML element - final Element rootGroupElement = (Element) rootElement.getElementsByTagName("rootGroup").item(0); - - // if this controller isn't initialized or its emtpy, add the root group, otherwise update - if (!initialized || existingFlowEmpty) { - logger.trace("Adding root process group"); - addProcessGroup(controller, /* parent group */ null, rootGroupElement, encryptor); - } else { - logger.trace("Updating root process group"); - updateProcessGroup(controller, /* parent group */ null, rootGroupElement, encryptor); + synchronized (configuration) { + // get the root element + final Element rootElement = (Element) configuration.getElementsByTagName("flowController").item(0); + + // set controller config + logger.trace("Updating flow config"); + final Integer maxThreadCount = getInteger(rootElement, "maxThreadCount"); + if (maxThreadCount == null) { + controller.setMaxTimerDrivenThreadCount(getInt(rootElement, "maxTimerDrivenThreadCount")); + controller.setMaxEventDrivenThreadCount(getInt(rootElement, "maxEventDrivenThreadCount")); + } else { + controller.setMaxTimerDrivenThreadCount(maxThreadCount * 2 / 3); + controller.setMaxEventDrivenThreadCount(maxThreadCount / 3); + } + + // get the root group XML element + final Element rootGroupElement = (Element) rootElement.getElementsByTagName("rootGroup").item(0); + + final Element controllerServicesElement = (Element) DomUtils.getChild(rootElement, "controllerServices"); + if ( controllerServicesElement != null ) { + final List serviceElements = DomUtils.getChildElementsByTagName(controllerServicesElement, "controllerService"); + + if ( !initialized || existingFlowEmpty ) { + ControllerServiceLoader.loadControllerServices(serviceElements, controller, encryptor, controller.getBulletinRepository(), autoResumeState); + } else { + for ( final Element serviceElement : serviceElements ) { + updateControllerService(controller, serviceElement, encryptor); + } + } + } + + // if this controller isn't initialized or its emtpy, add the root group, otherwise update + if (!initialized || existingFlowEmpty) { + logger.trace("Adding root process group"); + addProcessGroup(controller, /* parent group */ null, rootGroupElement, encryptor); + } else { + logger.trace("Updating root process group"); + updateProcessGroup(controller, /* parent group */ null, rootGroupElement, encryptor); + } + + final Element reportingTasksElement = (Element) DomUtils.getChild(rootElement, "reportingTasks"); + if ( reportingTasksElement != null ) { + final List taskElements = DomUtils.getChildElementsByTagName(reportingTasksElement, "reportingTask"); + for ( final Element taskElement : taskElements ) { + if ( !initialized || existingFlowEmpty ) { + addReportingTask(controller, taskElement, encryptor); + } else { + updateReportingTask(controller, taskElement, encryptor); + } + } + } } } - + logger.trace("Synching templates"); if ((existingTemplates == null || existingTemplates.length == 0) && proposedFlow.getTemplates() != null && proposedFlow.getTemplates().length > 0) { // need to load templates @@ -313,7 +370,124 @@ public class StandardFlowSynchronizer implements FlowSynchronizer { return baos.toByteArray(); } + + + private void updateControllerService(final FlowController controller, final Element controllerServiceElement, final StringEncryptor encryptor) { + final ControllerServiceDTO dto = FlowFromDOMFactory.getControllerService(controllerServiceElement, encryptor); + + final ControllerServiceState dtoState = ControllerServiceState.valueOf(dto.getState()); + final boolean dtoEnabled = (dtoState == ControllerServiceState.ENABLED || dtoState == ControllerServiceState.ENABLING); + + final ControllerServiceNode serviceNode = controller.getControllerServiceNode(dto.getId()); + final ControllerServiceState serviceState = serviceNode.getState(); + final boolean serviceEnabled = (serviceState == ControllerServiceState.ENABLED || serviceState == ControllerServiceState.ENABLING); + + if (dtoEnabled && !serviceEnabled) { + controller.enableControllerService(controller.getControllerServiceNode(dto.getId())); + } else if (!dtoEnabled && serviceEnabled) { + controller.disableControllerService(controller.getControllerServiceNode(dto.getId())); + } + } + + private void addReportingTask(final FlowController controller, final Element reportingTaskElement, final StringEncryptor encryptor) throws ReportingTaskInstantiationException { + final ReportingTaskDTO dto = FlowFromDOMFactory.getReportingTask(reportingTaskElement, encryptor); + + final ReportingTaskNode reportingTask = controller.createReportingTask(dto.getType(), dto.getId(), false); + reportingTask.setName(dto.getName()); + reportingTask.setComments(dto.getComments()); + reportingTask.setScheduldingPeriod(dto.getSchedulingPeriod()); + reportingTask.setSchedulingStrategy(SchedulingStrategy.valueOf(dto.getSchedulingStrategy())); + + reportingTask.setAnnotationData(dto.getAnnotationData()); + + for (final Map.Entry entry : dto.getProperties().entrySet()) { + if (entry.getValue() == null) { + reportingTask.removeProperty(entry.getKey()); + } else { + reportingTask.setProperty(entry.getKey(), entry.getValue()); + } + } + + final ComponentLog componentLog = new SimpleProcessLogger(dto.getId(), reportingTask.getReportingTask()); + final ReportingInitializationContext config = new StandardReportingInitializationContext(dto.getId(), dto.getName(), + SchedulingStrategy.valueOf(dto.getSchedulingStrategy()), dto.getSchedulingPeriod(), componentLog, controller); + + try { + reportingTask.getReportingTask().initialize(config); + } catch (final InitializationException ie) { + throw new ReportingTaskInstantiationException("Failed to initialize reporting task of type " + dto.getType(), ie); + } + + if ( autoResumeState ) { + if ( ScheduledState.RUNNING.name().equals(dto.getState()) ) { + try { + controller.startReportingTask(reportingTask); + } catch (final Exception e) { + logger.error("Failed to start {} due to {}", reportingTask, e); + if ( logger.isDebugEnabled() ) { + logger.error("", e); + } + controller.getBulletinRepository().addBulletin(BulletinFactory.createBulletin( + "Reporting Tasks", Severity.ERROR.name(), "Failed to start " + reportingTask + " due to " + e)); + } + } else if ( ScheduledState.DISABLED.name().equals(dto.getState()) ) { + try { + controller.disableReportingTask(reportingTask); + } catch (final Exception e) { + logger.error("Failed to mark {} as disabled due to {}", reportingTask, e); + if ( logger.isDebugEnabled() ) { + logger.error("", e); + } + controller.getBulletinRepository().addBulletin(BulletinFactory.createBulletin( + "Reporting Tasks", Severity.ERROR.name(), "Failed to mark " + reportingTask + " as disabled due to " + e)); + } + } + } + } + private void updateReportingTask(final FlowController controller, final Element reportingTaskElement, final StringEncryptor encryptor) { + final ReportingTaskDTO dto = FlowFromDOMFactory.getReportingTask(reportingTaskElement, encryptor); + final ReportingTaskNode taskNode = controller.getReportingTaskNode(dto.getId()); + + if (!taskNode.getScheduledState().name().equals(dto.getState())) { + try { + switch (ScheduledState.valueOf(dto.getState())) { + case DISABLED: + if ( taskNode.isRunning() ) { + controller.stopReportingTask(taskNode); + } + controller.disableReportingTask(taskNode); + break; + case RUNNING: + if ( taskNode.getScheduledState() == ScheduledState.DISABLED ) { + controller.enableReportingTask(taskNode); + } + controller.startReportingTask(taskNode); + break; + case STOPPED: + if (taskNode.getScheduledState() == ScheduledState.DISABLED) { + controller.enableReportingTask(taskNode); + } else if (taskNode.getScheduledState() == ScheduledState.RUNNING) { + controller.stopReportingTask(taskNode); + } + break; + } + } catch (final IllegalStateException ise) { + logger.error("Failed to change Scheduled State of {} from {} to {} due to {}", taskNode, taskNode.getScheduledState().name(), dto.getState(), ise.toString()); + logger.error("", ise); + + // create bulletin for the Processor Node + controller.getBulletinRepository().addBulletin(BulletinFactory.createBulletin("Node Reconnection", Severity.ERROR.name(), + "Failed to change Scheduled State of " + taskNode + " from " + taskNode.getScheduledState().name() + " to " + dto.getState() + " due to " + ise.toString())); + + // create bulletin at Controller level. + controller.getBulletinRepository().addBulletin(BulletinFactory.createBulletin("Node Reconnection", Severity.ERROR.name(), + "Failed to change Scheduled State of " + taskNode + " from " + taskNode.getScheduledState().name() + " to " + dto.getState() + " due to " + ise.toString())); + } + } + } + + private ProcessGroup updateProcessGroup(final FlowController controller, final ProcessGroup parentGroup, final Element processGroupElement, final StringEncryptor encryptor) throws ProcessorInstantiationException { // get the parent group ID diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardProcessorNode.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardProcessorNode.java index fe72ae45ca..355e303a61 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardProcessorNode.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardProcessorNode.java @@ -52,6 +52,7 @@ import org.apache.nifi.connectable.Connectable; import org.apache.nifi.connectable.ConnectableType; import org.apache.nifi.connectable.Connection; import org.apache.nifi.connectable.Position; +import org.apache.nifi.controller.service.ControllerServiceNode; import org.apache.nifi.controller.service.ControllerServiceProvider; import org.apache.nifi.groups.ProcessGroup; import org.apache.nifi.logging.LogLevel; @@ -120,7 +121,7 @@ public class StandardProcessorNode extends ProcessorNode implements Connectable private SchedulingStrategy schedulingStrategy; // guarded by read/write lock @SuppressWarnings("deprecation") - StandardProcessorNode(final Processor processor, final String uuid, final ValidationContextFactory validationContextFactory, + public StandardProcessorNode(final Processor processor, final String uuid, final ValidationContextFactory validationContextFactory, final ProcessScheduler scheduler, final ControllerServiceProvider controllerServiceProvider) { super(processor, uuid, validationContextFactory, controllerServiceProvider); @@ -985,6 +986,16 @@ public class StandardProcessorNode extends ProcessorNode implements Connectable readLock.unlock(); } } + + @Override + public int getActiveThreadCount() { + readLock.lock(); + try { + return processScheduler.getActiveThreadCount(this); + } finally { + readLock.unlock(); + } + } @Override public boolean isValid() { @@ -1182,8 +1193,13 @@ public class StandardProcessorNode extends ProcessorNode implements Connectable public void verifyCanStart() { readLock.lock(); try { - if (scheduledState.get() != ScheduledState.STOPPED) { - throw new IllegalStateException(this + " is not stopped"); + switch (getScheduledState()) { + case DISABLED: + throw new IllegalStateException(this + " cannot be started because it is disabled"); + case RUNNING: + throw new IllegalStateException(this + " cannot be started because it is already running"); + case STOPPED: + break; } verifyNoActiveThreads(); @@ -1194,6 +1210,31 @@ public class StandardProcessorNode extends ProcessorNode implements Connectable readLock.unlock(); } } + + @Override + public void verifyCanStart(final Set ignoredReferences) { + switch (getScheduledState()) { + case DISABLED: + throw new IllegalStateException(this + " cannot be started because it is disabled"); + case RUNNING: + throw new IllegalStateException(this + " cannot be started because it is already running"); + case STOPPED: + break; + } + verifyNoActiveThreads(); + + final Set ids = new HashSet<>(); + for ( final ControllerServiceNode node : ignoredReferences ) { + ids.add(node.getIdentifier()); + } + + final Collection validationResults = getValidationErrors(ids); + for ( final ValidationResult result : validationResults ) { + if ( !result.isValid() ) { + throw new IllegalStateException(this + " cannot be started because it is not valid: " + result); + } + } + } @Override public void verifyCanStop() { diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/TemplateManager.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/TemplateManager.java index aa095d132d..30d4365d34 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/TemplateManager.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/TemplateManager.java @@ -42,24 +42,25 @@ import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.ReadWriteLock; import java.util.concurrent.locks.ReentrantReadWriteLock; +import org.apache.commons.lang3.StringUtils; +import org.apache.nifi.persistence.TemplateDeserializer; +import org.apache.nifi.persistence.TemplateSerializer; import org.apache.nifi.stream.io.ByteArrayInputStream; import org.apache.nifi.stream.io.ByteArrayOutputStream; import org.apache.nifi.stream.io.DataOutputStream; import org.apache.nifi.stream.io.StreamUtils; -import org.apache.nifi.persistence.TemplateDeserializer; -import org.apache.nifi.persistence.TemplateSerializer; import org.apache.nifi.web.api.dto.ConnectableDTO; import org.apache.nifi.web.api.dto.ConnectionDTO; +import org.apache.nifi.web.api.dto.ControllerServiceDTO; import org.apache.nifi.web.api.dto.FlowSnippetDTO; import org.apache.nifi.web.api.dto.ProcessGroupDTO; import org.apache.nifi.web.api.dto.ProcessorConfigDTO; -import org.apache.nifi.web.api.dto.ProcessorConfigDTO.PropertyDescriptorDTO; import org.apache.nifi.web.api.dto.ProcessorDTO; +import org.apache.nifi.web.api.dto.PropertyDescriptorDTO; import org.apache.nifi.web.api.dto.RemoteProcessGroupContentsDTO; import org.apache.nifi.web.api.dto.RemoteProcessGroupDTO; import org.apache.nifi.web.api.dto.RemoteProcessGroupPortDTO; import org.apache.nifi.web.api.dto.TemplateDTO; -import org.apache.commons.lang3.StringUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -272,6 +273,11 @@ public class TemplateManager { if (snippet.getProcessGroups() != null) { scrubProcessGroups(snippet.getProcessGroups()); } + + // go through each controller service if specified + if (snippet.getControllerServices() != null) { + scrubControllerServices(snippet.getControllerServices()); + } } } @@ -315,7 +321,6 @@ public class TemplateManager { } } - processorConfig.setDescriptors(null); processorConfig.setCustomUiUrl(null); } @@ -323,6 +328,24 @@ public class TemplateManager { processorDTO.setValidationErrors(null); } } + + private void scrubControllerServices(final Set controllerServices) { + for ( final ControllerServiceDTO serviceDTO : controllerServices ) { + final Map properties = serviceDTO.getProperties(); + final Map descriptors = serviceDTO.getDescriptors(); + + if ( properties != null && descriptors != null ) { + for ( final PropertyDescriptorDTO descriptor : descriptors.values() ) { + if ( descriptor.isSensitive() ) { + properties.put(descriptor.getName(), null); + } + } + } + + serviceDTO.setCustomUiUrl(null); + serviceDTO.setValidationErrors(null); + } + } /** * Scrubs connections prior to saving. This includes removing available diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/reporting/AbstractReportingTaskNode.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/reporting/AbstractReportingTaskNode.java index 7c3734ad45..05b3a06e46 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/reporting/AbstractReportingTaskNode.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/reporting/AbstractReportingTaskNode.java @@ -16,11 +16,14 @@ */ package org.apache.nifi.controller.reporting; +import java.util.Collection; +import java.util.HashSet; +import java.util.Set; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicReference; +import org.apache.nifi.components.ValidationResult; import org.apache.nifi.controller.AbstractConfiguredComponent; -import org.apache.nifi.controller.Availability; import org.apache.nifi.controller.ConfigurationContext; import org.apache.nifi.controller.ControllerServiceLookup; import org.apache.nifi.controller.ProcessScheduler; @@ -29,6 +32,7 @@ import org.apache.nifi.controller.ScheduledState; import org.apache.nifi.controller.ValidationContextFactory; import org.apache.nifi.controller.annotation.OnConfigured; import org.apache.nifi.controller.exception.ProcessorLifeCycleException; +import org.apache.nifi.controller.service.ControllerServiceNode; import org.apache.nifi.controller.service.ControllerServiceProvider; import org.apache.nifi.controller.service.StandardConfigurationContext; import org.apache.nifi.nar.NarCloseable; @@ -45,8 +49,8 @@ public abstract class AbstractReportingTaskNode extends AbstractConfiguredCompon private final AtomicReference schedulingStrategy = new AtomicReference<>(SchedulingStrategy.TIMER_DRIVEN); private final AtomicReference schedulingPeriod = new AtomicReference<>("5 mins"); - private final AtomicReference availability = new AtomicReference<>(Availability.NODE_ONLY); - + + private volatile String comment; private volatile ScheduledState scheduledState = ScheduledState.STOPPED; public AbstractReportingTaskNode(final ReportingTask reportingTask, final String id, @@ -58,16 +62,6 @@ public abstract class AbstractReportingTaskNode extends AbstractConfiguredCompon this.serviceLookup = controllerServiceProvider; } - @Override - public Availability getAvailability() { - return availability.get(); - } - - @Override - public void setAvailability(final Availability availability) { - this.availability.set(availability); - } - @Override public void setSchedulingStrategy(final SchedulingStrategy schedulingStrategy) { this.schedulingStrategy.set(schedulingStrategy); @@ -102,6 +96,11 @@ public abstract class AbstractReportingTaskNode extends AbstractConfiguredCompon public boolean isRunning() { return processScheduler.isScheduled(this) || processScheduler.getActiveThreadCount(this) > 0; } + + @Override + public int getActiveThreadCount() { + return processScheduler.getActiveThreadCount(this); + } @Override public ConfigurationContext getConfigurationContext() { @@ -142,7 +141,8 @@ public abstract class AbstractReportingTaskNode extends AbstractConfiguredCompon return removed; } - private void onConfigured() { + @SuppressWarnings("deprecation") + private void onConfigured() { // We need to invoke any method annotation with the OnConfigured annotation in order to // maintain backward compatibility. This will be removed when we remove the old, deprecated annotations. try (final NarCloseable x = NarCloseable.withNarLoader()) { @@ -158,6 +158,16 @@ public abstract class AbstractReportingTaskNode extends AbstractConfiguredCompon } @Override + public String getComments() { + return comment; + } + + @Override + public void setComments(final String comment) { + this.comment = comment; + } + + @Override public void verifyCanDelete() { if (isRunning()) { throw new IllegalStateException("Cannot delete " + reportingTask + " because it is currently running"); @@ -207,4 +217,38 @@ public abstract class AbstractReportingTaskNode extends AbstractConfiguredCompon throw new IllegalStateException("Cannot update " + reportingTask + " because it is currently running"); } } + + @Override + public void verifyCanStart(final Set ignoredReferences) { + switch (getScheduledState()) { + case DISABLED: + throw new IllegalStateException(this + " cannot be started because it is disabled"); + case RUNNING: + throw new IllegalStateException(this + " cannot be started because it is already running"); + case STOPPED: + break; + } + final int activeThreadCount = getActiveThreadCount(); + if ( activeThreadCount > 0 ) { + throw new IllegalStateException(this + " cannot be started because it has " + activeThreadCount + " active threads already"); + } + + final Set ids = new HashSet<>(); + for ( final ControllerServiceNode node : ignoredReferences ) { + ids.add(node.getIdentifier()); + } + + final Collection validationResults = getValidationErrors(ids); + for ( final ValidationResult result : validationResults ) { + if ( !result.isValid() ) { + throw new IllegalStateException(this + " cannot be started because it is not valid: " + result); + } + } + } + + + @Override + public String toString() { + return "ReportingTask[id=" + getIdentifier() + ", name=" + getName() + "]"; + } } diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/reporting/StandardReportingContext.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/reporting/StandardReportingContext.java index ed48e20495..3d57533b2d 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/reporting/StandardReportingContext.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/reporting/StandardReportingContext.java @@ -124,9 +124,20 @@ public class StandardReportingContext implements ReportingContext, ControllerSer public boolean isControllerServiceEnabled(final String serviceIdentifier) { return serviceProvider.isControllerServiceEnabled(serviceIdentifier); } + + @Override + public boolean isControllerServiceEnabling(final String serviceIdentifier) { + return serviceProvider.isControllerServiceEnabling(serviceIdentifier); + } @Override public ControllerServiceLookup getControllerServiceLookup() { return this; } + + @Override + public String getControllerServiceName(final String serviceIdentifier) { + return serviceProvider.getControllerServiceName(serviceIdentifier); + } + } diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/reporting/StandardReportingInitializationContext.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/reporting/StandardReportingInitializationContext.java index d576f9c0c0..435dbd0582 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/reporting/StandardReportingInitializationContext.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/reporting/StandardReportingInitializationContext.java @@ -22,6 +22,7 @@ import java.util.concurrent.TimeUnit; import org.apache.nifi.controller.ControllerService; import org.apache.nifi.controller.ControllerServiceLookup; import org.apache.nifi.controller.service.ControllerServiceProvider; +import org.apache.nifi.logging.ComponentLog; import org.apache.nifi.reporting.ReportingInitializationContext; import org.apache.nifi.scheduling.SchedulingStrategy; import org.apache.nifi.util.FormatUtils; @@ -33,13 +34,16 @@ public class StandardReportingInitializationContext implements ReportingInitiali private final String schedulingPeriod; private final SchedulingStrategy schedulingStrategy; private final ControllerServiceProvider serviceProvider; - - public StandardReportingInitializationContext(final String id, final String name, final SchedulingStrategy schedulingStrategy, final String schedulingPeriod, final ControllerServiceProvider serviceProvider) { + private final ComponentLog logger; + + public StandardReportingInitializationContext(final String id, final String name, final SchedulingStrategy schedulingStrategy, + final String schedulingPeriod, final ComponentLog logger, final ControllerServiceProvider serviceProvider) { this.id = id; this.name = name; this.schedulingPeriod = schedulingPeriod; this.serviceProvider = serviceProvider; this.schedulingStrategy = schedulingStrategy; + this.logger = logger; } @Override @@ -89,8 +93,23 @@ public class StandardReportingInitializationContext implements ReportingInitiali return serviceProvider.isControllerServiceEnabled(serviceIdentifier); } + @Override + public boolean isControllerServiceEnabling(final String serviceIdentifier) { + return serviceProvider.isControllerServiceEnabling(serviceIdentifier); + } + @Override public ControllerServiceLookup getControllerServiceLookup() { return this; } + + @Override + public String getControllerServiceName(final String serviceIdentifier) { + return serviceProvider.getControllerServiceName(serviceIdentifier); + } + + @Override + public ComponentLog getLogger() { + return logger; + } } diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/StandardProcessScheduler.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/StandardProcessScheduler.java index 4407451a8a..89850cc165 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/StandardProcessScheduler.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/StandardProcessScheduler.java @@ -19,6 +19,8 @@ package org.apache.nifi.controller.scheduling; import static java.util.Objects.requireNonNull; import java.lang.reflect.InvocationTargetException; +import java.util.HashSet; +import java.util.Set; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.ExecutorService; @@ -32,20 +34,26 @@ import org.apache.nifi.annotation.lifecycle.OnEnabled; import org.apache.nifi.annotation.lifecycle.OnScheduled; import org.apache.nifi.annotation.lifecycle.OnStopped; import org.apache.nifi.annotation.lifecycle.OnUnscheduled; +import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.connectable.Connectable; import org.apache.nifi.connectable.Funnel; import org.apache.nifi.connectable.Port; import org.apache.nifi.controller.AbstractPort; import org.apache.nifi.controller.ConfigurationContext; +import org.apache.nifi.controller.ControllerService; import org.apache.nifi.controller.Heartbeater; import org.apache.nifi.controller.ProcessScheduler; import org.apache.nifi.controller.ProcessorNode; import org.apache.nifi.controller.ReportingTaskNode; import org.apache.nifi.controller.ScheduledState; import org.apache.nifi.controller.annotation.OnConfigured; +import org.apache.nifi.controller.service.ControllerServiceNode; import org.apache.nifi.controller.service.ControllerServiceProvider; +import org.apache.nifi.controller.service.ControllerServiceState; +import org.apache.nifi.controller.service.StandardConfigurationContext; import org.apache.nifi.encrypt.StringEncryptor; import org.apache.nifi.engine.FlowEngine; +import org.apache.nifi.logging.ComponentLog; import org.apache.nifi.logging.ProcessorLog; import org.apache.nifi.nar.NarCloseable; import org.apache.nifi.processor.SchedulingContext; @@ -144,6 +152,8 @@ public final class StandardProcessScheduler implements ProcessScheduler { componentLifeCycleThreadPool.shutdown(); } + + @Override public void schedule(final ReportingTaskNode taskNode) { final ScheduleState scheduleState = getScheduleState(requireNonNull(taskNode)); if (scheduleState.isScheduled()) { @@ -176,16 +186,11 @@ public final class StandardProcessScheduler implements ProcessScheduler { } break; - } catch (final InvocationTargetException ite) { - LOG.error("Failed to invoke the On-Scheduled Lifecycle methods of {} due to {}; administratively yielding this ReportingTask and will attempt to schedule it again after {}", - new Object[]{reportingTask, ite.getTargetException(), administrativeYieldDuration}); - LOG.error("", ite.getTargetException()); - - try { - Thread.sleep(administrativeYieldMillis); - } catch (final InterruptedException ie) { - } } catch (final Exception e) { + final Throwable cause = (e instanceof InvocationTargetException) ? e.getCause() : e; + final ComponentLog componentLog = new SimpleProcessLogger(reportingTask.getIdentifier(), reportingTask); + componentLog.error("Failed to invoke @OnEnabled method due to {}", cause); + LOG.error("Failed to invoke the On-Scheduled Lifecycle methods of {} due to {}; administratively yielding this ReportingTask and will attempt to schedule it again after {}", new Object[]{reportingTask, e.toString(), administrativeYieldDuration}, e); try { @@ -200,18 +205,23 @@ public final class StandardProcessScheduler implements ProcessScheduler { }; componentLifeCycleThreadPool.execute(startReportingTaskRunnable); + taskNode.setScheduledState(ScheduledState.RUNNING); } + + @Override public void unschedule(final ReportingTaskNode taskNode) { final ScheduleState scheduleState = getScheduleState(requireNonNull(taskNode)); if (!scheduleState.isScheduled()) { return; } - + + taskNode.verifyCanStop(); final SchedulingAgent agent = getSchedulingAgent(taskNode.getSchedulingStrategy()); final ReportingTask reportingTask = taskNode.getReportingTask(); scheduleState.setScheduled(false); - + taskNode.setScheduledState(ScheduledState.STOPPED); + final Runnable unscheduleReportingTaskRunnable = new Runnable() { @SuppressWarnings("deprecation") @Override @@ -222,18 +232,15 @@ public final class StandardProcessScheduler implements ProcessScheduler { try (final NarCloseable x = NarCloseable.withNarLoader()) { ReflectionUtils.invokeMethodsWithAnnotation(OnUnscheduled.class, org.apache.nifi.processor.annotation.OnUnscheduled.class, reportingTask, configurationContext); } - } catch (final InvocationTargetException ite) { - LOG.error("Failed to invoke the @OnConfigured methods of {} due to {}; administratively yielding this ReportingTask and will attempt to schedule it again after {}", - new Object[]{reportingTask, ite.getTargetException(), administrativeYieldDuration}); - LOG.error("", ite.getTargetException()); - - try { - Thread.sleep(administrativeYieldMillis); - } catch (final InterruptedException ie) { - } } catch (final Exception e) { - LOG.error("Failed to invoke the @OnConfigured methods of {} due to {}; administratively yielding this ReportingTask and will attempt to schedule it again after {}", - new Object[]{reportingTask, e.toString(), administrativeYieldDuration}, e); + final Throwable cause = (e instanceof InvocationTargetException) ? e.getCause() : e; + final ComponentLog componentLog = new SimpleProcessLogger(reportingTask.getIdentifier(), reportingTask); + componentLog.error("Failed to invoke @OnUnscheduled method due to {}", cause); + + LOG.error("Failed to invoke the @OnUnscheduled methods of {} due to {}; administratively yielding this ReportingTask and will attempt to schedule it again after {}", + reportingTask, cause.toString(), administrativeYieldDuration); + LOG.error("", cause); + try { Thread.sleep(administrativeYieldMillis); } catch (final InterruptedException ie) { @@ -274,20 +281,38 @@ public final class StandardProcessScheduler implements ProcessScheduler { } if (!procNode.isValid()) { - throw new IllegalStateException("Processor " + procNode.getName() + " is not in a valid state"); + throw new IllegalStateException("Processor " + procNode.getName() + " is not in a valid state due to " + procNode.getValidationErrors()); } final Runnable startProcRunnable = new Runnable() { - @SuppressWarnings("deprecation") @Override + @SuppressWarnings("deprecation") public void run() { try (final NarCloseable x = NarCloseable.withNarLoader()) { long lastStopTime = scheduleState.getLastStopTime(); final StandardProcessContext processContext = new StandardProcessContext(procNode, controllerServiceProvider, encryptor); - while (true) { + final Set serviceIds = new HashSet<>(); + for ( final PropertyDescriptor descriptor : processContext.getProperties().keySet() ) { + final Class serviceDefinition = descriptor.getControllerServiceDefinition(); + if ( serviceDefinition != null ) { + final String serviceId = processContext.getProperty(descriptor).getValue(); + serviceIds.add(serviceId); + } + } + + attemptOnScheduled: while (true) { try { synchronized (scheduleState) { + for ( final String serviceId : serviceIds ) { + final boolean enabled = processContext.isControllerServiceEnabled(serviceId); + if ( !enabled ) { + LOG.debug("Controller Service with ID {} is not yet enabled, so will not start {} yet", serviceId, procNode); + Thread.sleep(administrativeYieldMillis); + continue attemptOnScheduled; + } + } + // if no longer scheduled to run, then we're finished. This can happen, for example, // if the @OnScheduled method throws an Exception and the user stops the processor // while we're administratively yielded. @@ -308,11 +333,12 @@ public final class StandardProcessScheduler implements ProcessScheduler { return; } } catch (final Exception e) { + final Throwable cause = (e instanceof InvocationTargetException) ? e.getCause() : e; final ProcessorLog procLog = new SimpleProcessLogger(procNode.getIdentifier(), procNode.getProcessor()); procLog.error("{} failed to invoke @OnScheduled method due to {}; processor will not be scheduled to run for {}", - new Object[]{procNode.getProcessor(), e.getCause(), administrativeYieldDuration}, e.getCause()); - LOG.error("Failed to invoke @OnScheduled method due to {}", e.getCause().toString(), e.getCause()); + new Object[]{procNode.getProcessor(), cause.getCause(), administrativeYieldDuration}, cause.getCause()); + LOG.error("Failed to invoke @OnScheduled method due to {}", cause.getCause().toString(), cause.getCause()); ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnUnscheduled.class, procNode.getProcessor(), processContext); ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnStopped.class, procNode.getProcessor(), processContext); @@ -535,11 +561,6 @@ public final class StandardProcessScheduler implements ProcessScheduler { } procNode.setScheduledState(ScheduledState.STOPPED); - - try (final NarCloseable x = NarCloseable.withNarLoader()) { - final ProcessorLog processorLog = new SimpleProcessLogger(procNode.getIdentifier(), procNode.getProcessor()); - ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnEnabled.class, procNode.getProcessor(), processorLog); - } } @Override @@ -549,11 +570,6 @@ public final class StandardProcessScheduler implements ProcessScheduler { } procNode.setScheduledState(ScheduledState.DISABLED); - - try (final NarCloseable x = NarCloseable.withNarLoader()) { - final ProcessorLog processorLog = new SimpleProcessLogger(procNode.getIdentifier(), procNode.getProcessor()); - ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnDisabled.class, procNode.getProcessor(), processorLog); - } } public synchronized void enableReportingTask(final ReportingTaskNode taskNode) { @@ -562,10 +578,6 @@ public final class StandardProcessScheduler implements ProcessScheduler { } taskNode.setScheduledState(ScheduledState.STOPPED); - - try (final NarCloseable x = NarCloseable.withNarLoader()) { - ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnEnabled.class, taskNode.getReportingTask()); - } } public synchronized void disableReportingTask(final ReportingTaskNode taskNode) { @@ -574,10 +586,6 @@ public final class StandardProcessScheduler implements ProcessScheduler { } taskNode.setScheduledState(ScheduledState.DISABLED); - - try (final NarCloseable x = NarCloseable.withNarLoader()) { - ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnDisabled.class, taskNode.getReportingTask()); - } } @Override @@ -605,4 +613,114 @@ public final class StandardProcessScheduler implements ProcessScheduler { } return scheduleState; } + + @Override + public void enableControllerService(final ControllerServiceNode service) { + service.setState(ControllerServiceState.ENABLING); + final ScheduleState scheduleState = getScheduleState(service); + + final Runnable enableRunnable = new Runnable() { + @Override + public void run() { + try (final NarCloseable x = NarCloseable.withNarLoader()) { + long lastStopTime = scheduleState.getLastStopTime(); + final ConfigurationContext configContext = new StandardConfigurationContext(service, controllerServiceProvider); + + while (true) { + try { + synchronized (scheduleState) { + // if no longer enabled, then we're finished. This can happen, for example, + // if the @OnEnabled method throws an Exception and the user disables the service + // while we're administratively yielded. + // + // we also check if the schedule state's last stop time is equal to what it was before. + // if not, then means that the service has been disabled and enabled again, so we should just + // bail; another thread will be responsible for invoking the @OnEnabled methods. + if (!scheduleState.isScheduled() || scheduleState.getLastStopTime() != lastStopTime) { + return; + } + + ReflectionUtils.invokeMethodsWithAnnotation(OnEnabled.class, service.getControllerServiceImplementation(), configContext); + heartbeater.heartbeat(); + service.setState(ControllerServiceState.ENABLED); + return; + } + } catch (final Exception e) { + final Throwable cause = (e instanceof InvocationTargetException) ? e.getCause() : e; + + final ComponentLog componentLog = new SimpleProcessLogger(service.getIdentifier(), service); + componentLog.error("Failed to invoke @OnEnabled method due to {}", cause); + LOG.error("Failed to invoke @OnEnabled method of {} due to {}", service.getControllerServiceImplementation(), cause.toString()); + if ( LOG.isDebugEnabled() ) { + LOG.error("", cause); + } + + ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnDisabled.class, service.getControllerServiceImplementation(), configContext); + Thread.sleep(administrativeYieldMillis); + continue; + } + } + } catch (final Throwable t) { + final Throwable cause = (t instanceof InvocationTargetException) ? t.getCause() : t; + final ComponentLog componentLog = new SimpleProcessLogger(service.getIdentifier(), service); + componentLog.error("Failed to invoke @OnEnabled method due to {}", cause); + + LOG.error("Failed to invoke @OnEnabled method on {} due to {}", service.getControllerServiceImplementation(), cause.toString()); + if ( LOG.isDebugEnabled() ) { + LOG.error("", cause); + } + } + } + }; + + scheduleState.setScheduled(true); + componentLifeCycleThreadPool.execute(enableRunnable); + } + + @Override + public void disableControllerService(final ControllerServiceNode service) { + service.verifyCanDisable(); + + final ScheduleState state = getScheduleState(requireNonNull(service)); + final Runnable disableRunnable = new Runnable() { + @Override + public void run() { + synchronized (state) { + state.setScheduled(false); + } + + try (final NarCloseable x = NarCloseable.withNarLoader()) { + final ConfigurationContext configContext = new StandardConfigurationContext(service, controllerServiceProvider); + + while(true) { + try { + ReflectionUtils.invokeMethodsWithAnnotation(OnDisabled.class, service.getControllerServiceImplementation(), configContext); + heartbeater.heartbeat(); + service.setState(ControllerServiceState.DISABLED); + return; + } catch (final Exception e) { + final Throwable cause = (e instanceof InvocationTargetException) ? e.getCause() : e; + final ComponentLog componentLog = new SimpleProcessLogger(service.getIdentifier(), service); + componentLog.error("Failed to invoke @OnDisabled method due to {}", cause); + + LOG.error("Failed to invoke @OnDisabled method of {} due to {}", service.getControllerServiceImplementation(), cause.toString()); + if ( LOG.isDebugEnabled() ) { + LOG.error("", cause); + } + + ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnDisabled.class, service.getControllerServiceImplementation(), configContext); + try { + Thread.sleep(administrativeYieldMillis); + } catch (final InterruptedException ie) {} + + continue; + } + } + } + } + }; + + service.setState(ControllerServiceState.DISABLING); + componentLifeCycleThreadPool.execute(disableRunnable); + } } diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/ControllerServiceLoader.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/ControllerServiceLoader.java index db44b5f0d6..1fde670901 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/ControllerServiceLoader.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/ControllerServiceLoader.java @@ -17,30 +17,29 @@ package org.apache.nifi.controller.service; import java.io.BufferedInputStream; -import java.io.File; import java.io.IOException; import java.io.InputStream; -import java.net.URL; -import java.nio.file.Files; -import java.nio.file.Path; -import java.nio.file.StandardOpenOption; import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.HashSet; import java.util.List; +import java.util.Map; +import java.util.Set; -import javax.xml.XMLConstants; import javax.xml.parsers.DocumentBuilder; import javax.xml.parsers.DocumentBuilderFactory; import javax.xml.parsers.ParserConfigurationException; -import javax.xml.validation.Schema; -import javax.xml.validation.SchemaFactory; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; +import org.apache.nifi.controller.FlowFromDOMFactory; +import org.apache.nifi.encrypt.StringEncryptor; +import org.apache.nifi.reporting.BulletinRepository; import org.apache.nifi.util.DomUtils; -import org.apache.nifi.util.file.FileUtils; +import org.apache.nifi.web.api.dto.ControllerServiceDTO; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import org.w3c.dom.Document; import org.w3c.dom.Element; -import org.w3c.dom.NodeList; import org.xml.sax.SAXException; import org.xml.sax.SAXParseException; @@ -49,35 +48,14 @@ import org.xml.sax.SAXParseException; */ public class ControllerServiceLoader { - private static final Log logger = LogFactory.getLog(ControllerServiceLoader.class); + private static final Logger logger = LoggerFactory.getLogger(ControllerServiceLoader.class); - private final Path serviceConfigXmlPath; - public ControllerServiceLoader(final Path serviceConfigXmlPath) throws IOException { - final File serviceConfigXmlFile = serviceConfigXmlPath.toFile(); - if (!serviceConfigXmlFile.exists() || !serviceConfigXmlFile.canRead()) { - throw new IOException(serviceConfigXmlPath + " does not appear to exist or cannot be read. Cannot load configuration."); - } - - this.serviceConfigXmlPath = serviceConfigXmlPath; - } - - public List loadControllerServices(final ControllerServiceProvider provider) throws IOException { - final SchemaFactory schemaFactory = SchemaFactory.newInstance(XMLConstants.W3C_XML_SCHEMA_NS_URI); + public static List loadControllerServices(final ControllerServiceProvider provider, final InputStream serializedStream, final StringEncryptor encryptor, final BulletinRepository bulletinRepo, final boolean autoResumeState) throws IOException { final DocumentBuilderFactory documentBuilderFactory = DocumentBuilderFactory.newInstance(); - InputStream fis = null; - BufferedInputStream bis = null; documentBuilderFactory.setNamespaceAware(true); - final List services = new ArrayList<>(); - - try { - final URL configurationResource = this.getClass().getResource("/ControllerServiceConfiguration.xsd"); - if (configurationResource == null) { - throw new NullPointerException("Unable to load XML Schema for ControllerServiceConfiguration"); - } - final Schema schema = schemaFactory.newSchema(configurationResource); - documentBuilderFactory.setSchema(schema); + try (final InputStream in = new BufferedInputStream(serializedStream)) { final DocumentBuilder builder = documentBuilderFactory.newDocumentBuilder(); builder.setErrorHandler(new org.xml.sax.ErrorHandler() { @@ -109,43 +87,72 @@ public class ControllerServiceLoader { throw err; } }); - - //if controllerService.xml does not exist, create an empty file... - fis = Files.newInputStream(this.serviceConfigXmlPath, StandardOpenOption.READ); - bis = new BufferedInputStream(fis); - if (Files.size(this.serviceConfigXmlPath) > 0) { - final Document document = builder.parse(bis); - final NodeList servicesNodes = document.getElementsByTagName("services"); - final Element servicesElement = (Element) servicesNodes.item(0); - - final List serviceNodes = DomUtils.getChildElementsByTagName(servicesElement, "service"); - for (final Element serviceElement : serviceNodes) { - //get properties for the specific controller task - id, name, class, - //and schedulingPeriod must be set - final String serviceId = DomUtils.getChild(serviceElement, "identifier").getTextContent().trim(); - final String serviceClass = DomUtils.getChild(serviceElement, "class").getTextContent().trim(); - - //set the class to be used for the configured controller task - final ControllerServiceNode serviceNode = provider.createControllerService(serviceClass, serviceId, false); - - //optional task-specific properties - for (final Element optionalProperty : DomUtils.getChildElementsByTagName(serviceElement, "property")) { - final String name = optionalProperty.getAttribute("name").trim(); - final String value = optionalProperty.getTextContent().trim(); - serviceNode.setProperty(name, value); - } - - services.add(serviceNode); - provider.enableControllerService(serviceNode); - } - } + + final Document document = builder.parse(in); + final Element controllerServices = document.getDocumentElement(); + final List serviceElements = DomUtils.getChildElementsByTagName(controllerServices, "controllerService"); + return new ArrayList(loadControllerServices(serviceElements, provider, encryptor, bulletinRepo, autoResumeState)); } catch (SAXException | ParserConfigurationException sxe) { throw new IOException(sxe); - } finally { - FileUtils.closeQuietly(fis); - FileUtils.closeQuietly(bis); } + } + + public static Collection loadControllerServices(final List serviceElements, final ControllerServiceProvider provider, final StringEncryptor encryptor, final BulletinRepository bulletinRepo, final boolean autoResumeState) { + final Map nodeMap = new HashMap<>(); + for ( final Element serviceElement : serviceElements ) { + final ControllerServiceNode serviceNode = createControllerService(provider, serviceElement, encryptor); + // We need to clone the node because it will be used in a separate thread below, and + // Element is not thread-safe. + nodeMap.put(serviceNode, (Element) serviceElement.cloneNode(true)); + } + for ( final Map.Entry entry : nodeMap.entrySet() ) { + configureControllerService(entry.getKey(), entry.getValue(), encryptor); + } + + // Start services + if ( autoResumeState ) { + final Set nodesToEnable = new HashSet<>(); + + for ( final ControllerServiceNode node : nodeMap.keySet() ) { + final Element controllerServiceElement = nodeMap.get(node); - return services; + final ControllerServiceDTO dto; + synchronized (controllerServiceElement.getOwnerDocument()) { + dto = FlowFromDOMFactory.getControllerService(controllerServiceElement, encryptor); + } + + final ControllerServiceState state = ControllerServiceState.valueOf(dto.getState()); + if (state == ControllerServiceState.ENABLED) { + nodesToEnable.add(node); + } + } + + provider.enableControllerServices(nodesToEnable); + } + + return nodeMap.keySet(); + } + + + private static ControllerServiceNode createControllerService(final ControllerServiceProvider provider, final Element controllerServiceElement, final StringEncryptor encryptor) { + final ControllerServiceDTO dto = FlowFromDOMFactory.getControllerService(controllerServiceElement, encryptor); + + final ControllerServiceNode node = provider.createControllerService(dto.getType(), dto.getId(), false); + node.setName(dto.getName()); + node.setComments(dto.getComments()); + return node; + } + + private static void configureControllerService(final ControllerServiceNode node, final Element controllerServiceElement, final StringEncryptor encryptor) { + final ControllerServiceDTO dto = FlowFromDOMFactory.getControllerService(controllerServiceElement, encryptor); + node.setAnnotationData(dto.getAnnotationData()); + + for (final Map.Entry entry : dto.getProperties().entrySet()) { + if (entry.getValue() == null) { + node.removeProperty(entry.getKey()); + } else { + node.setProperty(entry.getKey(), entry.getValue()); + } + } } } diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceInitializationContext.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceInitializationContext.java index 8b5f27f2d8..8d46b05c88 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceInitializationContext.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceInitializationContext.java @@ -21,14 +21,17 @@ import java.util.Set; import org.apache.nifi.controller.ControllerService; import org.apache.nifi.controller.ControllerServiceInitializationContext; import org.apache.nifi.controller.ControllerServiceLookup; +import org.apache.nifi.logging.ComponentLog; public class StandardControllerServiceInitializationContext implements ControllerServiceInitializationContext, ControllerServiceLookup { private final String id; private final ControllerServiceProvider serviceProvider; + private final ComponentLog logger; - public StandardControllerServiceInitializationContext(final String identifier, final ControllerServiceProvider serviceProvider) { + public StandardControllerServiceInitializationContext(final String identifier, final ComponentLog logger, final ControllerServiceProvider serviceProvider) { this.id = identifier; + this.logger = logger; this.serviceProvider = serviceProvider; } @@ -61,4 +64,19 @@ public class StandardControllerServiceInitializationContext implements Controlle public boolean isControllerServiceEnabled(final ControllerService service) { return serviceProvider.isControllerServiceEnabled(service); } + + @Override + public boolean isControllerServiceEnabling(String serviceIdentifier) { + return serviceProvider.isControllerServiceEnabling(serviceIdentifier); + } + + @Override + public String getControllerServiceName(final String serviceIdentifier) { + return serviceProvider.getControllerServiceName(serviceIdentifier); + } + + @Override + public ComponentLog getLogger() { + return logger; + } } diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceNode.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceNode.java index 741caeca72..c8c7ec91d8 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceNode.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceNode.java @@ -16,16 +16,17 @@ */ package org.apache.nifi.controller.service; +import java.util.Collection; +import java.util.Collections; import java.util.HashSet; import java.util.Set; -import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicReference; import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.ReadWriteLock; import java.util.concurrent.locks.ReentrantReadWriteLock; +import org.apache.nifi.components.ValidationResult; import org.apache.nifi.controller.AbstractConfiguredComponent; -import org.apache.nifi.controller.Availability; import org.apache.nifi.controller.ConfigurationContext; import org.apache.nifi.controller.ConfiguredComponent; import org.apache.nifi.controller.ControllerService; @@ -41,14 +42,14 @@ public class StandardControllerServiceNode extends AbstractConfiguredComponent i private final ControllerService implementation; private final ControllerServiceProvider serviceProvider; - private final AtomicReference availability = new AtomicReference<>(Availability.NODE_ONLY); - private final AtomicBoolean disabled = new AtomicBoolean(true); + private final AtomicReference stateRef = new AtomicReference<>(ControllerServiceState.DISABLED); private final ReadWriteLock rwLock = new ReentrantReadWriteLock(); private final Lock readLock = rwLock.readLock(); private final Lock writeLock = rwLock.writeLock(); private final Set referencingComponents = new HashSet<>(); + private String comment; public StandardControllerServiceNode(final ControllerService proxiedControllerService, final ControllerService implementation, final String id, final ValidationContextFactory validationContextFactory, final ControllerServiceProvider serviceProvider) { @@ -58,38 +59,7 @@ public class StandardControllerServiceNode extends AbstractConfiguredComponent i this.serviceProvider = serviceProvider; } - @Override - public boolean isDisabled() { - return disabled.get(); - } - - @Override - public void setDisabled(final boolean disabled) { - if (!disabled && !isValid()) { - throw new IllegalStateException("Cannot enable Controller Service " + implementation + " because it is not valid"); - } - - if (disabled) { - // do not allow a Controller Service to be disabled if it's currently being used. - final Set runningRefs = getReferences().getRunningReferences(); - if (!runningRefs.isEmpty()) { - throw new IllegalStateException("Cannot disable Controller Service because it is referenced (either directly or indirectly) by " + runningRefs.size() + " different components that are currently running"); - } - } - - this.disabled.set(disabled); - } - - @Override - public Availability getAvailability() { - return availability.get(); - } - - @Override - public void setAvailability(final Availability availability) { - this.availability.set(availability); - } - + @Override public ControllerService getProxiedControllerService() { return proxedControllerService; @@ -132,7 +102,7 @@ public class StandardControllerServiceNode extends AbstractConfiguredComponent i @Override public void verifyModifiable() throws IllegalStateException { - if (!isDisabled()) { + if (getState() != ControllerServiceState.DISABLED) { throw new IllegalStateException("Cannot modify Controller Service configuration because it is currently enabled. Please disable the Controller Service first."); } } @@ -140,7 +110,6 @@ public class StandardControllerServiceNode extends AbstractConfiguredComponent i @Override public void setProperty(final String name, final String value) { super.setProperty(name, value); - onConfigured(); } @@ -166,31 +135,96 @@ public class StandardControllerServiceNode extends AbstractConfiguredComponent i @Override public void verifyCanDelete() { - if ( !isDisabled() ) { + if ( getState() != ControllerServiceState.DISABLED ) { throw new IllegalStateException(implementation + " cannot be deleted because it is not disabled"); } } @Override public void verifyCanDisable() { + verifyCanDisable(Collections.emptySet()); + } + + @Override + public void verifyCanDisable(final Set ignoreReferences) { + final ControllerServiceState state = getState(); + if ( state != ControllerServiceState.ENABLED && state != ControllerServiceState.ENABLING ) { + throw new IllegalStateException("Cannot disable " + getControllerServiceImplementation() + " because it is not enabled"); + } + final ControllerServiceReference references = getReferences(); - final int numRunning = references.getRunningReferences().size(); - if ( numRunning > 0 ) { - throw new IllegalStateException(implementation + " cannot be disabled because it is referenced by " + numRunning + " components that are currently running"); + + for ( final ConfiguredComponent activeReference : references.getActiveReferences() ) { + if ( !ignoreReferences.contains(activeReference) ) { + throw new IllegalStateException(implementation + " cannot be disabled because it is referenced by at least one component that is currently running"); + } } } @Override public void verifyCanEnable() { - if ( !isDisabled() ) { + if ( getState() != ControllerServiceState.DISABLED ) { throw new IllegalStateException(implementation + " cannot be enabled because it is not disabled"); } + + if ( !isValid() ) { + throw new IllegalStateException(implementation + " cannot be enabled because it is not valid: " + getValidationErrors()); + } + } + + @Override + public void verifyCanEnable(final Set ignoredReferences) { + if (getState() != ControllerServiceState.DISABLED) { + throw new IllegalStateException(implementation + " cannot be enabled because it is not disabled"); + } + + final Set ids = new HashSet<>(); + for ( final ControllerServiceNode node : ignoredReferences ) { + ids.add(node.getIdentifier()); + } + + final Collection validationResults = getValidationErrors(ids); + for ( final ValidationResult result : validationResults ) { + if ( !result.isValid() ) { + throw new IllegalStateException(implementation + " cannot be enabled because it is not valid: " + result); + } + } } @Override public void verifyCanUpdate() { - if ( !isDisabled() ) { + if ( getState() != ControllerServiceState.DISABLED ) { throw new IllegalStateException(implementation + " cannot be updated because it is not disabled"); } } + + @Override + public String getComments() { + readLock.lock(); + try { + return comment; + } finally { + readLock.unlock(); + } + } + + @Override + public void setComments(final String comment) { + writeLock.lock(); + try { + this.comment = comment; + } finally { + writeLock.unlock(); + } + } + + @Override + public ControllerServiceState getState() { + return stateRef.get(); + } + + @Override + public void setState(final ControllerServiceState state) { + this.stateRef.set(state); + } } diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceProvider.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceProvider.java index 7a8e22fd4a..dfbfca55c5 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceProvider.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceProvider.java @@ -23,26 +23,39 @@ import java.lang.reflect.InvocationTargetException; import java.lang.reflect.Method; import java.lang.reflect.Proxy; import java.util.ArrayList; +import java.util.Collection; import java.util.Collections; +import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; import org.apache.nifi.annotation.lifecycle.OnAdded; -import org.apache.nifi.annotation.lifecycle.OnEnabled; -import org.apache.nifi.annotation.lifecycle.OnDisabled; import org.apache.nifi.annotation.lifecycle.OnRemoved; +import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.controller.ConfigurationContext; +import org.apache.nifi.controller.ConfiguredComponent; import org.apache.nifi.controller.ControllerService; +import org.apache.nifi.controller.ProcessScheduler; +import org.apache.nifi.controller.ProcessorNode; +import org.apache.nifi.controller.ReportingTaskNode; +import org.apache.nifi.controller.ScheduledState; import org.apache.nifi.controller.ValidationContextFactory; -import org.apache.nifi.controller.exception.ControllerServiceAlreadyExistsException; -import org.apache.nifi.controller.exception.ControllerServiceNotFoundException; +import org.apache.nifi.controller.exception.ControllerServiceInstantiationException; import org.apache.nifi.controller.exception.ProcessorLifeCycleException; +import org.apache.nifi.events.BulletinFactory; +import org.apache.nifi.logging.ComponentLog; import org.apache.nifi.nar.ExtensionManager; import org.apache.nifi.nar.NarCloseable; +import org.apache.nifi.processor.SimpleProcessLogger; import org.apache.nifi.processor.StandardValidationContextFactory; +import org.apache.nifi.reporting.BulletinRepository; +import org.apache.nifi.reporting.Severity; import org.apache.nifi.util.ObjectHolder; import org.apache.nifi.util.ReflectionUtils; import org.slf4j.Logger; @@ -55,8 +68,10 @@ public class StandardControllerServiceProvider implements ControllerServiceProvi private static final Logger logger = LoggerFactory.getLogger(StandardControllerServiceProvider.class); - private final Map controllerServices; + private final ProcessScheduler processScheduler; + private final ConcurrentMap controllerServices; private static final Set validDisabledMethods; + private final BulletinRepository bulletinRepo; static { // methods that are okay to be called when the service is disabled. @@ -70,10 +85,12 @@ public class StandardControllerServiceProvider implements ControllerServiceProvi validDisabledMethods = Collections.unmodifiableSet(validMethods); } - public StandardControllerServiceProvider() { + public StandardControllerServiceProvider(final ProcessScheduler scheduler, final BulletinRepository bulletinRepo) { // the following 2 maps must be updated atomically, but we do not lock around them because they are modified // only in the createControllerService method, and both are modified before the method returns this.controllerServices = new ConcurrentHashMap<>(); + this.processScheduler = scheduler; + this.bulletinRepo = bulletinRepo; } private Class[] getInterfaces(final Class cls) { @@ -95,21 +112,24 @@ public class StandardControllerServiceProvider implements ControllerServiceProvi populateInterfaces(superClass, interfacesDefinedThusFar); } } - + @Override public ControllerServiceNode createControllerService(final String type, final String id, final boolean firstTimeAdded) { if (type == null || id == null) { throw new NullPointerException(); } - if (controllerServices.containsKey(id)) { - throw new ControllerServiceAlreadyExistsException(id); - } - + final ClassLoader currentContextClassLoader = Thread.currentThread().getContextClassLoader(); try { final ClassLoader cl = ExtensionManager.getClassLoader(type); - Thread.currentThread().setContextClassLoader(cl); - final Class rawClass = Class.forName(type, false, cl); + final Class rawClass; + if ( cl == null ) { + rawClass = Class.forName(type); + } else { + Thread.currentThread().setContextClassLoader(cl); + rawClass = Class.forName(type, false, cl); + } + final Class controllerServiceClass = rawClass.asSubclass(ControllerService.class); final ControllerService originalService = controllerServiceClass.newInstance(); @@ -124,7 +144,9 @@ public class StandardControllerServiceProvider implements ControllerServiceProvi } final ControllerServiceNode node = serviceNodeHolder.get(); - if (node.isDisabled() && !validDisabledMethods.contains(method)) { + final ControllerServiceState state = node.getState(); + final boolean disabled = (state != ControllerServiceState.ENABLED); // only allow method call if service state is ENABLED. + if (disabled && !validDisabledMethods.contains(method)) { // Use nar class loader here because we are implicitly calling toString() on the original implementation. try (final NarCloseable narCloseable = NarCloseable.withNarLoader()) { throw new IllegalStateException("Cannot invoke method " + method + " on Controller Service " + originalService + " because the Controller Service is disabled"); @@ -143,17 +165,22 @@ public class StandardControllerServiceProvider implements ControllerServiceProvi } }; - final ControllerService proxiedService = (ControllerService) Proxy.newProxyInstance(cl, getInterfaces(controllerServiceClass), invocationHandler); - logger.info("Loaded service {} as configured.", type); + final ControllerService proxiedService; + if ( cl == null ) { + proxiedService = (ControllerService) Proxy.newProxyInstance(getClass().getClassLoader(), getInterfaces(controllerServiceClass), invocationHandler); + } else { + proxiedService = (ControllerService) Proxy.newProxyInstance(cl, getInterfaces(controllerServiceClass), invocationHandler); + } + logger.info("Created Controller Service of type {} with identifier {}", type, id); - originalService.initialize(new StandardControllerServiceInitializationContext(id, this)); + final ComponentLog serviceLogger = new SimpleProcessLogger(id, originalService); + originalService.initialize(new StandardControllerServiceInitializationContext(id, serviceLogger, this)); final ValidationContextFactory validationContextFactory = new StandardValidationContextFactory(this); final ControllerServiceNode serviceNode = new StandardControllerServiceNode(proxiedService, originalService, id, validationContextFactory, this); serviceNodeHolder.set(serviceNode); - serviceNode.setAnnotationData(null); - serviceNode.setName(id); + serviceNode.setName(rawClass.getSimpleName()); if ( firstTimeAdded ) { try (final NarCloseable x = NarCloseable.withNarLoader()) { @@ -166,7 +193,7 @@ public class StandardControllerServiceProvider implements ControllerServiceProvi this.controllerServices.put(id, serviceNode); return serviceNode; } catch (final Throwable t) { - throw new ControllerServiceNotFoundException(t); + throw new ControllerServiceInstantiationException(t); } finally { if (currentContextClassLoader != null) { Thread.currentThread().setContextClassLoader(currentContextClassLoader); @@ -174,29 +201,242 @@ public class StandardControllerServiceProvider implements ControllerServiceProvi } } + + + @Override + public void disableReferencingServices(final ControllerServiceNode serviceNode) { + // Get a list of all Controller Services that need to be disabled, in the order that they need to be + // disabled. + final List toDisable = findRecursiveReferences(serviceNode, ControllerServiceNode.class); + final Set serviceSet = new HashSet<>(toDisable); + + for ( final ControllerServiceNode nodeToDisable : toDisable ) { + final ControllerServiceState state = nodeToDisable.getState(); + + if ( state != ControllerServiceState.DISABLED && state != ControllerServiceState.DISABLING ) { + nodeToDisable.verifyCanDisable(serviceSet); + } + } + + Collections.reverse(toDisable); + for ( final ControllerServiceNode nodeToDisable : toDisable ) { + final ControllerServiceState state = nodeToDisable.getState(); + + if ( state != ControllerServiceState.DISABLED && state != ControllerServiceState.DISABLING ) { + disableControllerService(nodeToDisable); + } + } + } + + + @Override + public void scheduleReferencingComponents(final ControllerServiceNode serviceNode) { + // find all of the schedulable components (processors, reporting tasks) that refer to this Controller Service, + // or a service that references this controller service, etc. + final List processors = findRecursiveReferences(serviceNode, ProcessorNode.class); + final List reportingTasks = findRecursiveReferences(serviceNode, ReportingTaskNode.class); + + // verify that we can start all components (that are not disabled) before doing anything + for ( final ProcessorNode node : processors ) { + if ( node.getScheduledState() != ScheduledState.DISABLED ) { + node.verifyCanStart(); + } + } + for ( final ReportingTaskNode node : reportingTasks ) { + if ( node.getScheduledState() != ScheduledState.DISABLED ) { + node.verifyCanStart(); + } + } + + // start all of the components that are not disabled + for ( final ProcessorNode node : processors ) { + if ( node.getScheduledState() != ScheduledState.DISABLED ) { + node.getProcessGroup().startProcessor(node); + } + } + for ( final ReportingTaskNode node : reportingTasks ) { + if ( node.getScheduledState() != ScheduledState.DISABLED ) { + processScheduler.schedule(node); + } + } + } + + @Override + public void unscheduleReferencingComponents(final ControllerServiceNode serviceNode) { + // find all of the schedulable components (processors, reporting tasks) that refer to this Controller Service, + // or a service that references this controller service, etc. + final List processors = findRecursiveReferences(serviceNode, ProcessorNode.class); + final List reportingTasks = findRecursiveReferences(serviceNode, ReportingTaskNode.class); + + // verify that we can stop all components (that are running) before doing anything + for ( final ProcessorNode node : processors ) { + if ( node.getScheduledState() == ScheduledState.RUNNING ) { + node.verifyCanStop(); + } + } + for ( final ReportingTaskNode node : reportingTasks ) { + if ( node.getScheduledState() == ScheduledState.RUNNING ) { + node.verifyCanStop(); + } + } + + // stop all of the components that are running + for ( final ProcessorNode node : processors ) { + if ( node.getScheduledState() == ScheduledState.RUNNING ) { + node.getProcessGroup().stopProcessor(node); + } + } + for ( final ReportingTaskNode node : reportingTasks ) { + if ( node.getScheduledState() == ScheduledState.RUNNING ) { + processScheduler.unschedule(node); + } + } + } + @Override public void enableControllerService(final ControllerServiceNode serviceNode) { serviceNode.verifyCanEnable(); - - try (final NarCloseable x = NarCloseable.withNarLoader()) { - final ConfigurationContext configContext = new StandardConfigurationContext(serviceNode, this); - ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnEnabled.class, serviceNode.getControllerServiceImplementation(), configContext); + processScheduler.enableControllerService(serviceNode); + } + + @Override + public void enableControllerServices(final Collection serviceNodes) { + final Set servicesToEnable = new HashSet<>(); + // Ensure that all nodes are already disabled + for ( final ControllerServiceNode serviceNode : serviceNodes ) { + final ControllerServiceState curState = serviceNode.getState(); + if ( ControllerServiceState.DISABLED.equals(curState) ) { + servicesToEnable.add(serviceNode); + } else { + logger.warn("Cannot enable {} because it is not disabled; current state is {}", serviceNode, curState); + } } - serviceNode.setDisabled(false); + // determine the order to load the services. We have to ensure that if service A references service B, then B + // is enabled first, and so on. + final Map idToNodeMap = new HashMap<>(); + for ( final ControllerServiceNode node : servicesToEnable ) { + idToNodeMap.put(node.getIdentifier(), node); + } + + // We can have many Controller Services dependent on one another. We can have many of these + // disparate lists of Controller Services that are dependent on one another. We refer to each + // of these as a branch. + final List> branches = determineEnablingOrder(idToNodeMap); + + if ( branches.isEmpty() ) { + logger.info("No Controller Services to enable"); + return; + } else { + logger.info("Will enable {} Controller Services", servicesToEnable.size()); + } + + // Mark all services that are configured to be enabled as 'ENABLING'. This allows Processors, reporting tasks + // to be valid so that they can be scheduled. + for ( final List branch : branches ) { + for ( final ControllerServiceNode nodeToEnable : branch ) { + nodeToEnable.setState(ControllerServiceState.ENABLING); + } + } + + final Set enabledNodes = Collections.synchronizedSet(new HashSet()); + final ExecutorService executor = Executors.newFixedThreadPool(Math.min(10, branches.size())); + for ( final List branch : branches ) { + final Runnable enableBranchRunnable = new Runnable() { + @Override + public void run() { + logger.debug("Enabling Controller Service Branch {}", branch); + + for ( final ControllerServiceNode serviceNode : branch ) { + try { + if ( !enabledNodes.contains(serviceNode) ) { + enabledNodes.add(serviceNode); + + logger.info("Enabling {}", serviceNode); + try { + processScheduler.enableControllerService(serviceNode); + } catch (final Exception e) { + logger.error("Failed to enable " + serviceNode + " due to " + e); + if ( logger.isDebugEnabled() ) { + logger.error("", e); + } + + if ( bulletinRepo != null ) { + bulletinRepo.addBulletin(BulletinFactory.createBulletin( + "Controller Service", Severity.ERROR.name(), "Could not start " + serviceNode + " due to " + e)); + } + } + } + + // wait for service to finish enabling. + while ( ControllerServiceState.ENABLING.equals(serviceNode.getState()) ) { + try { + Thread.sleep(100L); + } catch (final InterruptedException ie) {} + } + + logger.info("State for {} is now {}", serviceNode, serviceNode.getState()); + } catch (final Exception e) { + logger.error("Failed to enable {} due to {}", serviceNode, e.toString()); + if ( logger.isDebugEnabled() ) { + logger.error("", e); + } + } + } + } + }; + + executor.submit(enableBranchRunnable); + } + + executor.shutdown(); } + static List> determineEnablingOrder(final Map serviceNodeMap) { + final List> orderedNodeLists = new ArrayList<>(); + + for ( final ControllerServiceNode node : serviceNodeMap.values() ) { + if ( orderedNodeLists.contains(node) ) { + continue; // this node is already in the list. + } + + final List branch = new ArrayList<>(); + determineEnablingOrder(serviceNodeMap, node, branch, new HashSet()); + orderedNodeLists.add(branch); + } + + return orderedNodeLists; + } + + + private static void determineEnablingOrder(final Map serviceNodeMap, final ControllerServiceNode contextNode, final List orderedNodes, final Set visited) { + if ( visited.contains(contextNode) ) { + return; + } + + for ( final Map.Entry entry : contextNode.getProperties().entrySet() ) { + if ( entry.getKey().getControllerServiceDefinition() != null ) { + final String referencedServiceId = entry.getValue(); + if ( referencedServiceId != null ) { + final ControllerServiceNode referencedNode = serviceNodeMap.get(referencedServiceId); + if ( !orderedNodes.contains(referencedNode) ) { + visited.add(contextNode); + determineEnablingOrder(serviceNodeMap, referencedNode, orderedNodes, visited); + } + } + } + } + + if ( !orderedNodes.contains(contextNode) ) { + orderedNodes.add(contextNode); + } + } + + @Override public void disableControllerService(final ControllerServiceNode serviceNode) { serviceNode.verifyCanDisable(); - - // We must set the service to disabled before we invoke the OnDisabled methods because the service node - // can throw Exceptions if we attempt to disable the service while it's known to be in use. - serviceNode.setDisabled(true); - - try (final NarCloseable x = NarCloseable.withNarLoader()) { - ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnDisabled.class, serviceNode.getControllerServiceImplementation()); - } + processScheduler.disableControllerService(serviceNode); } @Override @@ -213,9 +453,15 @@ public class StandardControllerServiceProvider implements ControllerServiceProvi @Override public boolean isControllerServiceEnabled(final String serviceIdentifier) { final ControllerServiceNode node = controllerServices.get(serviceIdentifier); - return (node == null) ? false : !node.isDisabled(); + return (node == null) ? false : (ControllerServiceState.ENABLED == node.getState()); } + @Override + public boolean isControllerServiceEnabling(final String serviceIdentifier) { + final ControllerServiceNode node = controllerServices.get(serviceIdentifier); + return (node == null) ? false : (ControllerServiceState.ENABLING == node.getState()); + } + @Override public ControllerServiceNode getControllerServiceNode(final String serviceIdentifier) { return controllerServices.get(serviceIdentifier); @@ -234,6 +480,11 @@ public class StandardControllerServiceProvider implements ControllerServiceProvi } @Override + public String getControllerServiceName(final String serviceIdentifier) { + final ControllerServiceNode node = getControllerServiceNode(serviceIdentifier); + return node == null ? null : node.getName(); + } + public void removeControllerService(final ControllerServiceNode serviceNode) { final ControllerServiceNode existing = controllerServices.get(serviceNode.getIdentifier()); if ( existing == null || existing != serviceNode ) { @@ -247,6 +498,139 @@ public class StandardControllerServiceProvider implements ControllerServiceProvi ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnRemoved.class, serviceNode.getControllerServiceImplementation(), configurationContext); } + for ( final Map.Entry entry : serviceNode.getProperties().entrySet() ) { + final PropertyDescriptor descriptor = entry.getKey(); + if (descriptor.getControllerServiceDefinition() != null ) { + final String value = entry.getValue() == null ? descriptor.getDefaultValue() : entry.getValue(); + if ( value != null ) { + final ControllerServiceNode referencedNode = getControllerServiceNode(value); + if ( referencedNode != null ) { + referencedNode.removeReference(serviceNode); + } + } + } + } + controllerServices.remove(serviceNode.getIdentifier()); } + + @Override + public Set getAllControllerServices() { + return new HashSet<>(controllerServices.values()); + } + + + /** + * Returns a List of all components that reference the given referencedNode (either directly or indirectly through + * another service) that are also of the given componentType. The list that is returned is in the order in which they will + * need to be 'activated' (enabled/started). + * @param referencedNode + * @param componentType + * @return + */ + private List findRecursiveReferences(final ControllerServiceNode referencedNode, final Class componentType) { + final List references = new ArrayList<>(); + + for ( final ConfiguredComponent referencingComponent : referencedNode.getReferences().getReferencingComponents() ) { + if ( componentType.isAssignableFrom(referencingComponent.getClass()) ) { + references.add(componentType.cast(referencingComponent)); + } + + if ( referencingComponent instanceof ControllerServiceNode ) { + final ControllerServiceNode referencingNode = (ControllerServiceNode) referencingComponent; + + // find components recursively that depend on referencingNode. + final List recursive = findRecursiveReferences(referencingNode, componentType); + + // For anything that depends on referencing node, we want to add it to the list, but we know + // that it must come after the referencing node, so we first remove any existing occurrence. + references.removeAll(recursive); + references.addAll(recursive); + } + } + + return references; + } + + + @Override + public void enableReferencingServices(final ControllerServiceNode serviceNode) { + final List recursiveReferences = findRecursiveReferences(serviceNode, ControllerServiceNode.class); + enableReferencingServices(serviceNode, recursiveReferences); + } + + private void enableReferencingServices(final ControllerServiceNode serviceNode, final List recursiveReferences) { + if ( serviceNode.getState() != ControllerServiceState.ENABLED && serviceNode.getState() != ControllerServiceState.ENABLING ) { + serviceNode.verifyCanEnable(new HashSet<>(recursiveReferences)); + } + + final Set ifEnabled = new HashSet<>(); + final List toEnable = findRecursiveReferences(serviceNode, ControllerServiceNode.class); + for ( final ControllerServiceNode nodeToEnable : toEnable ) { + final ControllerServiceState state = nodeToEnable.getState(); + if ( state != ControllerServiceState.ENABLED && state != ControllerServiceState.ENABLING ) { + nodeToEnable.verifyCanEnable(ifEnabled); + ifEnabled.add(nodeToEnable); + } + } + + for ( final ControllerServiceNode nodeToEnable : toEnable ) { + final ControllerServiceState state = nodeToEnable.getState(); + if ( state != ControllerServiceState.ENABLED && state != ControllerServiceState.ENABLING ) { + enableControllerService(nodeToEnable); + } + } + } + + @Override + public void verifyCanEnableReferencingServices(final ControllerServiceNode serviceNode) { + final List referencingServices = findRecursiveReferences(serviceNode, ControllerServiceNode.class); + final Set referencingServiceSet = new HashSet<>(referencingServices); + + for ( final ControllerServiceNode referencingService : referencingServices ) { + referencingService.verifyCanEnable(referencingServiceSet); + } + } + + @Override + public void verifyCanScheduleReferencingComponents(final ControllerServiceNode serviceNode) { + final List referencingServices = findRecursiveReferences(serviceNode, ControllerServiceNode.class); + final List referencingReportingTasks = findRecursiveReferences(serviceNode, ReportingTaskNode.class); + final List referencingProcessors = findRecursiveReferences(serviceNode, ProcessorNode.class); + + final Set referencingServiceSet = new HashSet<>(referencingServices); + + for ( final ReportingTaskNode taskNode : referencingReportingTasks ) { + if ( taskNode.getScheduledState() != ScheduledState.DISABLED ) { + taskNode.verifyCanStart(referencingServiceSet); + } + } + + for ( final ProcessorNode procNode : referencingProcessors ) { + if ( procNode.getScheduledState() != ScheduledState.DISABLED ) { + procNode.verifyCanStart(referencingServiceSet); + } + } + } + + @Override + public void verifyCanDisableReferencingServices(final ControllerServiceNode serviceNode) { + // Get a list of all Controller Services that need to be disabled, in the order that they need to be + // disabled. + final List toDisable = findRecursiveReferences(serviceNode, ControllerServiceNode.class); + final Set serviceSet = new HashSet<>(toDisable); + + for ( final ControllerServiceNode nodeToDisable : toDisable ) { + final ControllerServiceState state = nodeToDisable.getState(); + + if ( state != ControllerServiceState.DISABLED && state != ControllerServiceState.DISABLING ) { + nodeToDisable.verifyCanDisable(serviceSet); + } + } + } + + @Override + public void verifyCanStopReferencingComponents(final ControllerServiceNode serviceNode) { + // we can always stop referencing components + } } diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceReference.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceReference.java index a1c4984cc4..c470b99056 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceReference.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceReference.java @@ -58,23 +58,28 @@ public class StandardControllerServiceReference implements ControllerServiceRefe } @Override - public Set getRunningReferences() { - final Set runningReferences = new HashSet<>(); + public Set getActiveReferences() { + final Set activeReferences = new HashSet<>(); final Set serviceNodes = new HashSet<>(); for (final ConfiguredComponent component : components) { if (component instanceof ControllerServiceNode) { serviceNodes.add((ControllerServiceNode) component); + + final ControllerServiceState state = ((ControllerServiceNode) component).getState(); + if ( state != ControllerServiceState.DISABLED ) { + activeReferences.add(component); + } } else if (isRunning(component)) { - runningReferences.add(component); + activeReferences.add(component); } } - runningReferences.addAll(getRunningIndirectReferences(serviceNodes)); - return runningReferences; + activeReferences.addAll(getActiveIndirectReferences(serviceNodes)); + return activeReferences; } - private Set getRunningIndirectReferences(final Set referencingServices) { + private Set getActiveIndirectReferences(final Set referencingServices) { if (referencingServices.isEmpty()) { return Collections.emptySet(); } @@ -92,7 +97,7 @@ public class StandardControllerServiceReference implements ControllerServiceRefe } } - references.addAll(getRunningIndirectReferences(serviceNodes)); + references.addAll(getActiveIndirectReferences(serviceNodes)); } return references; diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/tasks/ReportingTaskWrapper.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/tasks/ReportingTaskWrapper.java index 9b7058101f..0c472c8fe3 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/tasks/ReportingTaskWrapper.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/tasks/ReportingTaskWrapper.java @@ -19,15 +19,13 @@ package org.apache.nifi.controller.tasks; import org.apache.nifi.annotation.lifecycle.OnStopped; import org.apache.nifi.controller.ReportingTaskNode; import org.apache.nifi.controller.scheduling.ScheduleState; +import org.apache.nifi.logging.ComponentLog; import org.apache.nifi.nar.NarCloseable; +import org.apache.nifi.processor.SimpleProcessLogger; import org.apache.nifi.util.ReflectionUtils; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; public class ReportingTaskWrapper implements Runnable { - private static final Logger logger = LoggerFactory.getLogger(ReportingTaskWrapper.class); - private final ReportingTaskNode taskNode; private final ScheduleState scheduleState; @@ -43,20 +41,23 @@ public class ReportingTaskWrapper implements Runnable { try (final NarCloseable narCloseable = NarCloseable.withNarLoader()) { taskNode.getReportingTask().onTrigger(taskNode.getReportingContext()); } catch (final Throwable t) { - logger.error("Error running task {} due to {}", taskNode.getReportingTask(), t.toString()); - if (logger.isDebugEnabled()) { - logger.error("", t); + final ComponentLog componentLog = new SimpleProcessLogger(taskNode.getIdentifier(), taskNode.getReportingTask()); + componentLog.error("Error running task {} due to {}", new Object[] {taskNode.getReportingTask(), t.toString()}); + if (componentLog.isDebugEnabled()) { + componentLog.error("", t); } } finally { - // if the processor is no longer scheduled to run and this is the last thread, - // invoke the OnStopped methods - if (!scheduleState.isScheduled() && scheduleState.getActiveThreadCount() == 1 && scheduleState.mustCallOnStoppedMethods()) { - try (final NarCloseable x = NarCloseable.withNarLoader()) { - ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnStopped.class, org.apache.nifi.processor.annotation.OnStopped.class, taskNode.getReportingTask(), taskNode.getReportingContext()); + try { + // if the reporting task is no longer scheduled to run and this is the last thread, + // invoke the OnStopped methods + if (!scheduleState.isScheduled() && scheduleState.getActiveThreadCount() == 1 && scheduleState.mustCallOnStoppedMethods()) { + try (final NarCloseable x = NarCloseable.withNarLoader()) { + ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnStopped.class, org.apache.nifi.processor.annotation.OnStopped.class, taskNode.getReportingTask(), taskNode.getConfigurationContext()); + } } + } finally { + scheduleState.decrementActiveThreadCount(); } - - scheduleState.decrementActiveThreadCount(); } } diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/fingerprint/FingerprintFactory.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/fingerprint/FingerprintFactory.java index 85755691f2..9a8ad28270 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/fingerprint/FingerprintFactory.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/fingerprint/FingerprintFactory.java @@ -41,12 +41,14 @@ import javax.xml.validation.SchemaFactory; import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.controller.FlowController; +import org.apache.nifi.controller.FlowFromDOMFactory; import org.apache.nifi.controller.Template; import org.apache.nifi.controller.exception.ProcessorInstantiationException; import org.apache.nifi.encrypt.StringEncryptor; import org.apache.nifi.processor.Processor; import org.apache.nifi.util.DomUtils; import org.apache.nifi.web.api.dto.ConnectionDTO; +import org.apache.nifi.web.api.dto.ControllerServiceDTO; import org.apache.nifi.web.api.dto.FlowSnippetDTO; import org.apache.nifi.web.api.dto.FunnelDTO; import org.apache.nifi.web.api.dto.LabelDTO; @@ -58,6 +60,7 @@ import org.apache.nifi.web.api.dto.ProcessorDTO; import org.apache.nifi.web.api.dto.RemoteProcessGroupContentsDTO; import org.apache.nifi.web.api.dto.RemoteProcessGroupDTO; import org.apache.nifi.web.api.dto.RemoteProcessGroupPortDTO; +import org.apache.nifi.web.api.dto.ReportingTaskDTO; import org.apache.nifi.web.api.dto.TemplateDTO; import org.apache.commons.lang3.StringUtils; import org.slf4j.Logger; @@ -250,6 +253,22 @@ public final class FingerprintFactory { // root group final Element rootGroupElem = (Element) DomUtils.getChildNodesByTagName(flowControllerElem, "rootGroup").item(0); addProcessGroupFingerprint(builder, rootGroupElem, controller); + + final Element controllerServicesElem = DomUtils.getChild(flowControllerElem, "controllerServices"); + if ( controllerServicesElem != null ) { + for ( final Element serviceElem : DomUtils.getChildElementsByTagName(controllerServicesElem, "controllerService") ) { + addControllerServiceFingerprint(builder, serviceElem); + } + } + + final Element reportingTasksElem = DomUtils.getChild(flowControllerElem, "reportingTasks"); + if ( reportingTasksElem != null ) { + for ( final Element taskElem : DomUtils.getChildElementsByTagName(reportingTasksElem, "reportingTask") ) { + addReportingTaskFingerprint(builder, taskElem); + } + } + + return builder; } @@ -832,6 +851,66 @@ public final class FingerprintFactory { builder.append(funnel.getId()); return builder; } + + private void addControllerServiceFingerprint(final StringBuilder builder, final Element controllerServiceElem) { + final ControllerServiceDTO dto = FlowFromDOMFactory.getControllerService(controllerServiceElem, encryptor); + addControllerServiceFingerprint(builder, dto); + } + + private void addControllerServiceFingerprint(final StringBuilder builder, final ControllerServiceDTO dto) { + builder.append(dto.getId()); + builder.append(dto.getType()); + builder.append(dto.getName()); + builder.append(dto.getComments()); + builder.append(dto.getAnnotationData()); + + final Map properties = dto.getProperties(); + if (properties == null) { + builder.append("NO_PROPERTIES"); + } else { + final SortedMap sortedProps = new TreeMap<>(properties); + for (final Map.Entry entry : sortedProps.entrySet()) { + final String propName = entry.getKey(); + final String propValue = entry.getValue(); + if (propValue == null) { + continue; + } + + builder.append(propName).append("=").append(propValue); + } + } + } + + private void addReportingTaskFingerprint(final StringBuilder builder, final Element element) { + final ReportingTaskDTO dto = FlowFromDOMFactory.getReportingTask(element, encryptor); + addReportingTaskFingerprint(builder, dto); + } + + private void addReportingTaskFingerprint(final StringBuilder builder, final ReportingTaskDTO dto) { + builder.append(dto.getId()); + builder.append(dto.getType()); + builder.append(dto.getName()); + builder.append(dto.getComments()); + builder.append(dto.getSchedulingPeriod()); + builder.append(dto.getSchedulingStrategy()); + builder.append(dto.getAnnotationData()); + + final Map properties = dto.getProperties(); + if (properties == null) { + builder.append("NO_PROPERTIES"); + } else { + final SortedMap sortedProps = new TreeMap<>(properties); + for (final Map.Entry entry : sortedProps.entrySet()) { + final String propName = entry.getKey(); + final String propValue = entry.getValue(); + if (propValue == null) { + continue; + } + + builder.append(propName).append("=").append(propValue); + } + } + } private Comparator getIdsComparator() { return new Comparator() { diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/groups/StandardProcessGroup.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/groups/StandardProcessGroup.java index 3cd5853265..216d015491 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/groups/StandardProcessGroup.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/groups/StandardProcessGroup.java @@ -30,8 +30,13 @@ import java.util.concurrent.atomic.AtomicReference; import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.ReentrantReadWriteLock; +import org.apache.commons.lang3.StringUtils; +import org.apache.commons.lang3.builder.HashCodeBuilder; +import org.apache.commons.lang3.builder.ToStringBuilder; +import org.apache.commons.lang3.builder.ToStringStyle; import org.apache.nifi.annotation.lifecycle.OnRemoved; import org.apache.nifi.annotation.lifecycle.OnShutdown; +import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.connectable.Connectable; import org.apache.nifi.connectable.ConnectableType; import org.apache.nifi.connectable.Connection; @@ -45,7 +50,9 @@ import org.apache.nifi.controller.ScheduledState; import org.apache.nifi.controller.Snippet; import org.apache.nifi.controller.exception.ProcessorLifeCycleException; import org.apache.nifi.controller.label.Label; +import org.apache.nifi.controller.service.ControllerServiceNode; import org.apache.nifi.controller.service.ControllerServiceProvider; +import org.apache.nifi.encrypt.StringEncryptor; import org.apache.nifi.logging.LogRepositoryFactory; import org.apache.nifi.nar.NarCloseable; import org.apache.nifi.processor.StandardProcessContext; @@ -53,11 +60,6 @@ import org.apache.nifi.remote.RemoteGroupPort; import org.apache.nifi.remote.RootGroupPort; import org.apache.nifi.util.NiFiProperties; import org.apache.nifi.util.ReflectionUtils; -import org.apache.commons.lang3.StringUtils; -import org.apache.commons.lang3.builder.HashCodeBuilder; -import org.apache.commons.lang3.builder.ToStringBuilder; -import org.apache.commons.lang3.builder.ToStringStyle; -import org.apache.nifi.encrypt.StringEncryptor; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -329,7 +331,8 @@ public final class StandardProcessGroup implements ProcessGroup { private void shutdown(final ProcessGroup procGroup) { for (final ProcessorNode node : procGroup.getProcessors()) { try (final NarCloseable x = NarCloseable.withNarLoader()) { - ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnShutdown.class, org.apache.nifi.processor.annotation.OnShutdown.class, node.getProcessor()); + final StandardProcessContext processContext = new StandardProcessContext(node, controllerServiceProvider, encryptor); + ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnShutdown.class, org.apache.nifi.processor.annotation.OnShutdown.class, node.getProcessor(), processContext); } } @@ -674,6 +677,19 @@ public final class StandardProcessGroup implements ProcessGroup { throw new ProcessorLifeCycleException("Failed to invoke 'OnRemoved' methods of " + processor, e); } + for ( final Map.Entry entry : processor.getProperties().entrySet() ) { + final PropertyDescriptor descriptor = entry.getKey(); + if (descriptor.getControllerServiceDefinition() != null ) { + final String value = entry.getValue() == null ? descriptor.getDefaultValue() : entry.getValue(); + if ( value != null ) { + final ControllerServiceNode serviceNode = controllerServiceProvider.getControllerServiceNode(value); + if ( serviceNode != null ) { + serviceNode.removeReference(processor); + } + } + } + } + processors.remove(id); LogRepositoryFactory.getRepository(processor.getIdentifier()).removeAllObservers(); diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/persistence/FlowConfigurationDAO.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/persistence/FlowConfigurationDAO.java index 4f3afaf598..8957314f9c 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/persistence/FlowConfigurationDAO.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/persistence/FlowConfigurationDAO.java @@ -19,15 +19,12 @@ package org.apache.nifi.persistence; import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; -import java.util.List; import org.apache.nifi.cluster.protocol.DataFlow; import org.apache.nifi.controller.FlowController; import org.apache.nifi.controller.FlowSerializationException; import org.apache.nifi.controller.FlowSynchronizationException; -import org.apache.nifi.controller.ReportingTaskNode; import org.apache.nifi.controller.UninheritableFlowException; -import org.apache.nifi.controller.service.ControllerServiceNode; /** * Interface to define service methods for FlowController configuration. @@ -110,26 +107,4 @@ public interface FlowConfigurationDAO { */ void save(FlowController flow, boolean archive) throws IOException; - /** - * Instantiates and schedules all controller tasks from the file used in the - * constructor - * - * @param controller - * @return - * @throws java.io.IOException - * @returns all of the ReportingTasks that were instantiated & scheduled - */ - List loadReportingTasks(FlowController controller) throws IOException; - - /** - * Instantiates all controller services from the file used in the - * constructor - * - * @param controller - * @return - * @throws java.io.IOException - * @returns all of the ReportingTasks that were instantiated & scheduled - */ - List loadControllerServices(FlowController controller) throws IOException; - } diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/persistence/StandardXMLFlowConfigurationDAO.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/persistence/StandardXMLFlowConfigurationDAO.java index c11aa72c78..b93ae8a678 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/persistence/StandardXMLFlowConfigurationDAO.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/persistence/StandardXMLFlowConfigurationDAO.java @@ -21,72 +21,36 @@ import java.io.File; import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; -import java.net.URL; import java.nio.file.Files; import java.nio.file.Path; import java.nio.file.StandardOpenOption; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Objects; import java.util.zip.GZIPInputStream; import java.util.zip.GZIPOutputStream; -import javax.xml.XMLConstants; -import javax.xml.parsers.DocumentBuilder; -import javax.xml.parsers.DocumentBuilderFactory; -import javax.xml.parsers.ParserConfigurationException; -import javax.xml.transform.dom.DOMSource; -import javax.xml.validation.Schema; -import javax.xml.validation.SchemaFactory; -import javax.xml.validation.Validator; - import org.apache.nifi.cluster.protocol.DataFlow; -import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.controller.FlowController; import org.apache.nifi.controller.FlowSerializationException; import org.apache.nifi.controller.FlowSynchronizationException; import org.apache.nifi.controller.FlowSynchronizer; -import org.apache.nifi.controller.ReportingTaskNode; import org.apache.nifi.controller.StandardFlowSerializer; import org.apache.nifi.controller.StandardFlowSynchronizer; import org.apache.nifi.controller.UninheritableFlowException; -import org.apache.nifi.controller.reporting.ReportingTaskInstantiationException; -import org.apache.nifi.controller.reporting.StandardReportingInitializationContext; -import org.apache.nifi.controller.service.ControllerServiceLoader; -import org.apache.nifi.controller.service.ControllerServiceNode; import org.apache.nifi.encrypt.StringEncryptor; -import org.apache.nifi.util.file.FileUtils; -import org.apache.nifi.nar.NarCloseable; -import org.apache.nifi.reporting.InitializationException; -import org.apache.nifi.reporting.ReportingInitializationContext; -import org.apache.nifi.reporting.ReportingTask; -import org.apache.nifi.scheduling.SchedulingStrategy; -import org.apache.nifi.util.DomUtils; import org.apache.nifi.util.NiFiProperties; - +import org.apache.nifi.util.file.FileUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.w3c.dom.DOMException; -import org.w3c.dom.Document; -import org.w3c.dom.Element; -import org.w3c.dom.NodeList; -import org.xml.sax.SAXException; -import org.xml.sax.SAXParseException; public final class StandardXMLFlowConfigurationDAO implements FlowConfigurationDAO { public static final String CONFIGURATION_ARCHIVE_DIR_KEY = "nifi.flow.configuration.archive.dir"; private final Path flowXmlPath; - private final Path taskConfigXmlPath; - private final ControllerServiceLoader servicerLoader; private final StringEncryptor encryptor; private static final Logger LOG = LoggerFactory.getLogger(StandardXMLFlowConfigurationDAO.class); - public StandardXMLFlowConfigurationDAO(final Path flowXml, final Path taskConfigXml, final Path serviceConfigXml, final StringEncryptor encryptor) throws IOException { + public StandardXMLFlowConfigurationDAO(final Path flowXml, final StringEncryptor encryptor) throws IOException { final File flowXmlFile = flowXml.toFile(); if (!flowXmlFile.exists()) { Files.createDirectories(flowXml.getParent()); @@ -96,14 +60,7 @@ public final class StandardXMLFlowConfigurationDAO implements FlowConfigurationD throw new IOException(flowXml + " exists but you have insufficient read/write privileges"); } - final File taskConfigXmlFile = Objects.requireNonNull(taskConfigXml).toFile(); - if ((!taskConfigXmlFile.exists() || !taskConfigXmlFile.canRead())) { - throw new IOException(taskConfigXml + " does not appear to exist or cannot be read. Cannot load configuration."); - } - this.flowXmlPath = flowXml; - this.taskConfigXmlPath = taskConfigXml; - this.servicerLoader = new ControllerServiceLoader(serviceConfigXml); this.encryptor = encryptor; } @@ -198,148 +155,4 @@ public final class StandardXMLFlowConfigurationDAO implements FlowConfigurationD } } - @Override - public List loadReportingTasks(final FlowController controller) { - final List tasks = new ArrayList<>(); - if (taskConfigXmlPath == null) { - LOG.info("No reporting tasks to start"); - return tasks; - } - - try { - final URL schemaUrl = getClass().getResource("/ReportingTaskConfiguration.xsd"); - final Document document = parse(taskConfigXmlPath.toFile(), schemaUrl); - - final NodeList tasksNodes = document.getElementsByTagName("tasks"); - final Element tasksElement = (Element) tasksNodes.item(0); - - for (final Element taskElement : DomUtils.getChildElementsByTagName(tasksElement, "task")) { - //add global properties common to all tasks - Map properties = new HashMap<>(); - - //get properties for the specific reporting task - id, name, class, - //and schedulingPeriod must be set - final String taskId = DomUtils.getChild(taskElement, "id").getTextContent().trim(); - final String taskName = DomUtils.getChild(taskElement, "name").getTextContent().trim(); - - final List schedulingStrategyNodeList = DomUtils.getChildElementsByTagName(taskElement, "schedulingStrategy"); - String schedulingStrategyValue = SchedulingStrategy.TIMER_DRIVEN.name(); - if (schedulingStrategyNodeList.size() == 1) { - final String specifiedValue = schedulingStrategyNodeList.get(0).getTextContent(); - - try { - schedulingStrategyValue = SchedulingStrategy.valueOf(specifiedValue).name(); - } catch (final Exception e) { - throw new RuntimeException("Cannot start Reporting Task with id " + taskId + " because its Scheduling Strategy does not have a valid value", e); - } - } - - final SchedulingStrategy schedulingStrategy = SchedulingStrategy.valueOf(schedulingStrategyValue); - final String taskSchedulingPeriod = DomUtils.getChild(taskElement, "schedulingPeriod").getTextContent().trim(); - final String taskClass = DomUtils.getChild(taskElement, "class").getTextContent().trim(); - - //optional task-specific properties - for (final Element optionalProperty : DomUtils.getChildElementsByTagName(taskElement, "property")) { - final String name = optionalProperty.getAttribute("name"); - final String value = optionalProperty.getTextContent().trim(); - properties.put(name, value); - } - - //set the class to be used for the configured reporting task - final ReportingTaskNode reportingTaskNode; - try { - reportingTaskNode = controller.createReportingTask(taskClass, taskId); - } catch (final ReportingTaskInstantiationException e) { - LOG.error("Unable to load reporting task {} due to {}", new Object[]{taskId, e}); - if (LOG.isDebugEnabled()) { - LOG.error("", e); - } - continue; - } - - reportingTaskNode.setName(taskName); - reportingTaskNode.setScheduldingPeriod(taskSchedulingPeriod); - reportingTaskNode.setSchedulingStrategy(schedulingStrategy); - - final ReportingTask reportingTask = reportingTaskNode.getReportingTask(); - - final ReportingInitializationContext config = new StandardReportingInitializationContext(taskId, taskName, schedulingStrategy, taskSchedulingPeriod, controller); - reportingTask.initialize(config); - - final Map resolvedProps; - try (final NarCloseable narCloseable = NarCloseable.withNarLoader()) { - resolvedProps = new HashMap<>(); - for (final Map.Entry entry : properties.entrySet()) { - final PropertyDescriptor descriptor = reportingTask.getPropertyDescriptor(entry.getKey()); - resolvedProps.put(descriptor, entry.getValue()); - } - } - - for (final Map.Entry entry : resolvedProps.entrySet()) { - reportingTaskNode.setProperty(entry.getKey().getName(), entry.getValue()); - } - - tasks.add(reportingTaskNode); - controller.startReportingTask(reportingTaskNode); - } - } catch (final SAXException | ParserConfigurationException | IOException | DOMException | NumberFormatException | InitializationException t) { - LOG.error("Unable to load reporting tasks from {} due to {}", new Object[]{taskConfigXmlPath, t}); - if (LOG.isDebugEnabled()) { - LOG.error("", t); - } - } - - return tasks; - } - - @Override - public List loadControllerServices(final FlowController controller) throws IOException { - return servicerLoader.loadControllerServices(controller); - } - - private Document parse(final File xmlFile, final URL schemaUrl) throws SAXException, ParserConfigurationException, IOException { - final SchemaFactory schemaFactory = SchemaFactory.newInstance(XMLConstants.W3C_XML_SCHEMA_NS_URI); - final Schema schema = schemaFactory.newSchema(schemaUrl); - final DocumentBuilderFactory docFactory = DocumentBuilderFactory.newInstance(); - docFactory.setSchema(schema); - final DocumentBuilder builder = docFactory.newDocumentBuilder(); - - builder.setErrorHandler(new org.xml.sax.ErrorHandler() { - @Override - public void fatalError(final SAXParseException err) throws SAXException { - LOG.error("Config file line " + err.getLineNumber() + ", col " + err.getColumnNumber() + ", uri " + err.getSystemId() + " :message: " + err.getMessage()); - if (LOG.isDebugEnabled()) { - LOG.error("Error Stack Dump", err); - } - throw err; - } - - @Override - public void error(final SAXParseException err) throws SAXParseException { - LOG.error("Config file line " + err.getLineNumber() + ", col " + err.getColumnNumber() + ", uri " + err.getSystemId() + " :message: " + err.getMessage()); - if (LOG.isDebugEnabled()) { - LOG.error("Error Stack Dump", err); - } - throw err; - } - - @Override - public void warning(final SAXParseException err) throws SAXParseException { - LOG.warn(" Config file line " + err.getLineNumber() + ", uri " + err.getSystemId() + " : message : " + err.getMessage()); - if (LOG.isDebugEnabled()) { - LOG.warn("Warning stack dump", err); - } - throw err; - } - }); - - // build the docuemnt - final Document document = builder.parse(xmlFile); - - // ensure schema compliance - final Validator validator = schema.newValidator(); - validator.validate(new DOMSource(document)); - - return document; - } } diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/processor/SimpleProcessLogger.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/processor/SimpleProcessLogger.java index 88f1790c22..25d8f10fe3 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/processor/SimpleProcessLogger.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/processor/SimpleProcessLogger.java @@ -28,16 +28,16 @@ public class SimpleProcessLogger implements ProcessorLog { private final Logger logger; private final LogRepository logRepository; - private final Processor processor; + private final Object component; - public SimpleProcessLogger(final String processorId, final Processor processor) { - this.logger = LoggerFactory.getLogger(processor.getClass()); - this.logRepository = LogRepositoryFactory.getRepository(processorId); - this.processor = processor; + public SimpleProcessLogger(final String componentId, final Object component) { + this.logger = LoggerFactory.getLogger(component.getClass()); + this.logRepository = LogRepositoryFactory.getRepository(componentId); + this.component = component; } private Object[] addProcessor(final Object[] originalArgs) { - return prependToArgs(originalArgs, processor); + return prependToArgs(originalArgs, component); } private Object[] prependToArgs(final Object[] originalArgs, final Object... toAdd) { @@ -63,7 +63,7 @@ public class SimpleProcessLogger implements ProcessorLog { @Override public void warn(final String msg, final Throwable t) { - warn("{} " + msg, new Object[]{processor}, t); + warn("{} " + msg, new Object[]{component}, t); } @Override @@ -93,15 +93,15 @@ public class SimpleProcessLogger implements ProcessorLog { @Override public void warn(String msg) { msg = "{} " + msg; - final Object[] os = {processor}; - logger.warn(msg, processor); + final Object[] os = {component}; + logger.warn(msg, component); logRepository.addLogMessage(LogLevel.WARN, msg, os); } @Override public void trace(String msg, Throwable t) { msg = "{} " + msg; - final Object[] os = {processor}; + final Object[] os = {component}; logger.trace(msg, os, t); logRepository.addLogMessage(LogLevel.TRACE, msg, os, t); } @@ -117,7 +117,7 @@ public class SimpleProcessLogger implements ProcessorLog { @Override public void trace(String msg) { msg = "{} " + msg; - final Object[] os = {processor}; + final Object[] os = {component}; logger.trace(msg, os); logRepository.addLogMessage(LogLevel.TRACE, msg, os); } @@ -160,7 +160,7 @@ public class SimpleProcessLogger implements ProcessorLog { @Override public void info(String msg, Throwable t) { msg = "{} " + msg; - final Object[] os = {processor}; + final Object[] os = {component}; logger.info(msg, os); if (logger.isDebugEnabled()) { @@ -181,7 +181,7 @@ public class SimpleProcessLogger implements ProcessorLog { @Override public void info(String msg) { msg = "{} " + msg; - final Object[] os = {processor}; + final Object[] os = {component}; logger.info(msg, os); logRepository.addLogMessage(LogLevel.INFO, msg, os); @@ -207,7 +207,7 @@ public class SimpleProcessLogger implements ProcessorLog { @Override public void error(String msg, Throwable t) { msg = "{} " + msg; - final Object[] os = {processor}; + final Object[] os = {component}; logger.error(msg, os, t); if (logger.isDebugEnabled()) { @@ -231,7 +231,7 @@ public class SimpleProcessLogger implements ProcessorLog { @Override public void error(String msg) { msg = "{} " + msg; - final Object[] os = {processor}; + final Object[] os = {component}; logger.error(msg, os); logRepository.addLogMessage(LogLevel.ERROR, msg, os); @@ -239,7 +239,7 @@ public class SimpleProcessLogger implements ProcessorLog { private Object[] addProcessorAndThrowable(final Object[] os, final Throwable t) { final Object[] modifiedArgs = new Object[os.length + 2]; - modifiedArgs[0] = processor.toString(); + modifiedArgs[0] = component.toString(); for (int i = 0; i < os.length; i++) { modifiedArgs[i + 1] = os[i]; } @@ -263,7 +263,7 @@ public class SimpleProcessLogger implements ProcessorLog { @Override public void debug(String msg, Throwable t) { msg = "{} " + msg; - final Object[] os = {processor}; + final Object[] os = {component}; logger.debug(msg, os, t); logRepository.addLogMessage(LogLevel.DEBUG, msg, os, t); @@ -298,7 +298,7 @@ public class SimpleProcessLogger implements ProcessorLog { @Override public void debug(String msg) { msg = "{} " + msg; - final Object[] os = {processor}; + final Object[] os = {component}; logger.debug(msg, os); logRepository.addLogMessage(LogLevel.DEBUG, msg, os); diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/processor/StandardProcessContext.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/processor/StandardProcessContext.java index cd0d31c465..d14a459099 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/processor/StandardProcessContext.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/processor/StandardProcessContext.java @@ -141,6 +141,11 @@ public class StandardProcessContext implements ProcessContext, ControllerService return controllerServiceProvider.isControllerServiceEnabled(serviceIdentifier); } + @Override + public boolean isControllerServiceEnabling(final String serviceIdentifier) { + return controllerServiceProvider.isControllerServiceEnabling(serviceIdentifier); + } + @Override public ControllerServiceLookup getControllerServiceLookup() { return this; @@ -170,4 +175,9 @@ public class StandardProcessContext implements ProcessContext, ControllerService return set; } + @Override + public String getControllerServiceName(final String serviceIdentifier) { + return controllerServiceProvider.getControllerServiceName(serviceIdentifier); + } + } diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/processor/StandardSchedulingContext.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/processor/StandardSchedulingContext.java index ac585044cd..c37a80d6bb 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/processor/StandardSchedulingContext.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/processor/StandardSchedulingContext.java @@ -25,6 +25,7 @@ import org.apache.nifi.controller.ControllerServiceLookup; import org.apache.nifi.controller.ProcessorNode; import org.apache.nifi.controller.service.ControllerServiceNode; import org.apache.nifi.controller.service.ControllerServiceProvider; +import org.apache.nifi.controller.service.ControllerServiceState; public class StandardSchedulingContext implements SchedulingContext { @@ -45,8 +46,8 @@ public class StandardSchedulingContext implements SchedulingContext { throw new IllegalArgumentException("Cannot lease Controller Service because no Controller Service exists with identifier " + identifier); } - if (serviceNode.isDisabled()) { - throw new IllegalStateException("Cannot lease Controller Service because Controller Service " + serviceNode.getProxiedControllerService() + " is currently disabled"); + if ( serviceNode.getState() != ControllerServiceState.ENABLED ) { + throw new IllegalStateException("Cannot lease Controller Service because Controller Service " + serviceNode.getProxiedControllerService() + " is not currently enabled"); } if (!serviceNode.isValid()) { diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/processor/StandardValidationContext.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/processor/StandardValidationContext.java index 99322be30c..c51cb9aa25 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/processor/StandardValidationContext.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/processor/StandardValidationContext.java @@ -20,6 +20,7 @@ import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Set; import org.apache.nifi.attribute.expression.language.PreparedQuery; import org.apache.nifi.attribute.expression.language.Query; @@ -41,11 +42,17 @@ public class StandardValidationContext implements ValidationContext { private final Map preparedQueries; private final Map expressionLanguageSupported; private final String annotationData; + private final Set serviceIdentifiersToNotValidate; public StandardValidationContext(final ControllerServiceProvider controllerServiceProvider, final Map properties, final String annotationData) { + this(controllerServiceProvider, Collections.emptySet(), properties, annotationData); + } + + public StandardValidationContext(final ControllerServiceProvider controllerServiceProvider, final Set serviceIdentifiersToNotValidate, final Map properties, final String annotationData) { this.controllerServiceProvider = controllerServiceProvider; this.properties = new HashMap<>(properties); this.annotationData = annotationData; + this.serviceIdentifiersToNotValidate = serviceIdentifiersToNotValidate; preparedQueries = new HashMap<>(properties.size()); for (final Map.Entry entry : properties.entrySet()) { @@ -101,6 +108,11 @@ public class StandardValidationContext implements ValidationContext { public ControllerServiceLookup getControllerServiceLookup() { return controllerServiceProvider; } + + @Override + public boolean isValidationRequired(final ControllerService service) { + return !serviceIdentifiersToNotValidate.contains(service.getIdentifier()); + } @Override public boolean isExpressionLanguagePresent(final String value) { diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/processor/StandardValidationContextFactory.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/processor/StandardValidationContextFactory.java index e172f935ea..c3df987032 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/processor/StandardValidationContextFactory.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/processor/StandardValidationContextFactory.java @@ -17,6 +17,7 @@ package org.apache.nifi.processor; import java.util.Map; +import java.util.Set; import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.components.ValidationContext; @@ -36,4 +37,8 @@ public class StandardValidationContextFactory implements ValidationContextFactor return new StandardValidationContext(serviceProvider, properties, annotationData); } + @Override + public ValidationContext newValidationContext(final Set serviceIdentifiersToNotValidate, final Map properties, final String annotationData) { + return new StandardValidationContext(serviceProvider, serviceIdentifiersToNotValidate, properties, annotationData); + } } diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/util/DomUtils.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/util/DomUtils.java index c8b65731de..da4f04deb6 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/util/DomUtils.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/util/DomUtils.java @@ -25,6 +25,16 @@ import org.w3c.dom.NodeList; public class DomUtils { + public static String getChildText(final Element element, final String tagName) { + final Element childElement = getChild(element, tagName); + if ( childElement == null ) { + return null; + } + + final String text = childElement.getTextContent(); + return (text == null) ? null : text.trim(); + } + public static Element getChild(final Element element, final String tagName) { final List children = getChildElementsByTagName(element, tagName); if (children.isEmpty()) { diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/resources/ControllerServiceConfiguration.xsd b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/resources/ControllerServiceConfiguration.xsd deleted file mode 100644 index d3efed19b6..0000000000 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/resources/ControllerServiceConfiguration.xsd +++ /dev/null @@ -1,61 +0,0 @@ - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/resources/FlowConfiguration.xsd b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/resources/FlowConfiguration.xsd index 1e6c25c8a1..00c71ac4ad 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/resources/FlowConfiguration.xsd +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/resources/FlowConfiguration.xsd @@ -28,6 +28,10 @@ + + + + @@ -58,11 +62,6 @@ IFF schedulingStrategy is EVENT_DRIVEN --> - @@ -332,4 +331,44 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + \ No newline at end of file diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/resources/ReportingTaskConfiguration.xsd b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/resources/ReportingTaskConfiguration.xsd deleted file mode 100644 index dcf10904a0..0000000000 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/resources/ReportingTaskConfiguration.xsd +++ /dev/null @@ -1,87 +0,0 @@ - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/StandardControllerServiceProviderTest.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/StandardControllerServiceProviderTest.java index b889bc81c1..7fef7065ff 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/StandardControllerServiceProviderTest.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/StandardControllerServiceProviderTest.java @@ -43,7 +43,7 @@ public class StandardControllerServiceProviderTest { public void setup() throws Exception { String id = "id"; String clazz = "org.apache.nifi.controller.service.util.TestControllerService"; - ControllerServiceProvider provider = new StandardControllerServiceProvider(); + ControllerServiceProvider provider = new StandardControllerServiceProvider(null, null); ControllerServiceNode node = provider.createControllerService(clazz,id,true); proxied = node.getProxiedControllerService(); implementation = node.getControllerServiceImplementation(); diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/TestStandardControllerServiceProvider.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/TestStandardControllerServiceProvider.java new file mode 100644 index 0000000000..3dc1752c9a --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/TestStandardControllerServiceProvider.java @@ -0,0 +1,385 @@ +/* + * 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.controller.service; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.UUID; + +import org.apache.nifi.controller.ProcessScheduler; +import org.apache.nifi.controller.ProcessorNode; +import org.apache.nifi.controller.ScheduledState; +import org.apache.nifi.controller.StandardProcessorNode; +import org.apache.nifi.controller.service.mock.DummyProcessor; +import org.apache.nifi.controller.service.mock.ServiceA; +import org.apache.nifi.controller.service.mock.ServiceB; +import org.apache.nifi.groups.ProcessGroup; +import org.apache.nifi.processor.StandardProcessorInitializationContext; +import org.apache.nifi.processor.StandardValidationContextFactory; +import org.junit.Assert; +import org.junit.Test; +import org.mockito.Mockito; +import org.mockito.invocation.InvocationOnMock; +import org.mockito.stubbing.Answer; + +public class TestStandardControllerServiceProvider { + + private ProcessScheduler createScheduler() { + final ProcessScheduler scheduler = Mockito.mock(ProcessScheduler.class); + Mockito.doAnswer(new Answer() { + @Override + public Object answer(final InvocationOnMock invocation) throws Throwable { + final ControllerServiceNode node = (ControllerServiceNode) invocation.getArguments()[0]; + node.verifyCanEnable(); + node.setState(ControllerServiceState.ENABLED); + return null; + } + }).when(scheduler).enableControllerService(Mockito.any(ControllerServiceNode.class)); + + Mockito.doAnswer(new Answer() { + @Override + public Object answer(final InvocationOnMock invocation) throws Throwable { + final ControllerServiceNode node = (ControllerServiceNode) invocation.getArguments()[0]; + node.verifyCanDisable(); + node.setState(ControllerServiceState.DISABLED); + return null; + } + }).when(scheduler).disableControllerService(Mockito.any(ControllerServiceNode.class)); + + return scheduler; + } + + @Test + public void testDisableControllerService() { + final ProcessScheduler scheduler = createScheduler(); + final StandardControllerServiceProvider provider = new StandardControllerServiceProvider(scheduler, null); + + final ControllerServiceNode serviceNode = provider.createControllerService(ServiceB.class.getName(), "B", false); + provider.enableControllerService(serviceNode); + provider.disableControllerService(serviceNode); + } + + @Test + public void testEnableDisableWithReference() { + final ProcessScheduler scheduler = createScheduler(); + final StandardControllerServiceProvider provider = new StandardControllerServiceProvider(scheduler, null); + + final ControllerServiceNode serviceNodeB = provider.createControllerService(ServiceB.class.getName(), "B", false); + final ControllerServiceNode serviceNodeA = provider.createControllerService(ServiceA.class.getName(), "A", false); + + serviceNodeA.setProperty(ServiceA.OTHER_SERVICE.getName(), "B"); + + try { + provider.enableControllerService(serviceNodeA); + Assert.fail("Was able to enable Service A but Service B is disabled."); + } catch (final IllegalStateException expected) { + } + + provider.enableControllerService(serviceNodeB); + provider.enableControllerService(serviceNodeA); + + try { + provider.disableControllerService(serviceNodeB); + Assert.fail("Was able to disable Service B but Service A is enabled and references B"); + } catch (final IllegalStateException expected) { + } + + provider.disableControllerService(serviceNodeA); + provider.disableControllerService(serviceNodeB); + } + + + @Test + public void testEnableReferencingServicesGraph() { + final ProcessScheduler scheduler = createScheduler(); + final StandardControllerServiceProvider provider = new StandardControllerServiceProvider(scheduler, null); + + // build a graph of controller services with dependencies as such: + // + // A -> B -> D + // C ---^----^ + // + // In other words, A references B, which references D. + // AND + // C references B and D. + // + // So we have to verify that if D is enabled, when we enable its referencing services, + // we enable C and B, even if we attempt to enable C before B... i.e., if we try to enable C, we cannot do so + // until B is first enabled so ensure that we enable B first. + + final ControllerServiceNode serviceNode1 = provider.createControllerService(ServiceA.class.getName(), "1", false); + final ControllerServiceNode serviceNode2 = provider.createControllerService(ServiceA.class.getName(), "2", false); + final ControllerServiceNode serviceNode3 = provider.createControllerService(ServiceA.class.getName(), "3", false); + final ControllerServiceNode serviceNode4 = provider.createControllerService(ServiceB.class.getName(), "4", false); + + serviceNode1.setProperty(ServiceA.OTHER_SERVICE.getName(), "2"); + serviceNode2.setProperty(ServiceA.OTHER_SERVICE.getName(), "4"); + serviceNode3.setProperty(ServiceA.OTHER_SERVICE.getName(), "2"); + serviceNode3.setProperty(ServiceA.OTHER_SERVICE_2.getName(), "4"); + + provider.enableControllerService(serviceNode4); + provider.enableReferencingServices(serviceNode4); + + assertEquals(ControllerServiceState.ENABLED, serviceNode3.getState()); + assertEquals(ControllerServiceState.ENABLED, serviceNode2.getState()); + assertEquals(ControllerServiceState.ENABLED, serviceNode1.getState()); + } + + + @Test + public void testStartStopReferencingComponents() { + final ProcessScheduler scheduler = createScheduler(); + final StandardControllerServiceProvider provider = new StandardControllerServiceProvider(scheduler, null); + + // build a graph of reporting tasks and controller services with dependencies as such: + // + // Processor P1 -> A -> B -> D + // Processor P2 -> C ---^----^ + // + // In other words, Processor P1 references Controller Service A, which references B, which references D. + // AND + // Processor P2 references Controller Service C, which references B and D. + // + // So we have to verify that if D is enabled, when we enable its referencing services, + // we enable C and B, even if we attempt to enable C before B... i.e., if we try to enable C, we cannot do so + // until B is first enabled so ensure that we enable B first. + + final ControllerServiceNode serviceNode1 = provider.createControllerService(ServiceA.class.getName(), "1", false); + final ControllerServiceNode serviceNode2 = provider.createControllerService(ServiceA.class.getName(), "2", false); + final ControllerServiceNode serviceNode3 = provider.createControllerService(ServiceA.class.getName(), "3", false); + final ControllerServiceNode serviceNode4 = provider.createControllerService(ServiceB.class.getName(), "4", false); + + final ProcessGroup mockProcessGroup = Mockito.mock(ProcessGroup.class); + Mockito.doAnswer(new Answer() { + @Override + public Object answer(InvocationOnMock invocation) throws Throwable { + final ProcessorNode procNode = (ProcessorNode) invocation.getArguments()[0]; + procNode.verifyCanStart(); + procNode.setScheduledState(ScheduledState.RUNNING); + return null; + } + }).when(mockProcessGroup).startProcessor(Mockito.any(ProcessorNode.class)); + + Mockito.doAnswer(new Answer() { + @Override + public Object answer(final InvocationOnMock invocation) throws Throwable { + final ProcessorNode procNode = (ProcessorNode) invocation.getArguments()[0]; + procNode.verifyCanStop(); + procNode.setScheduledState(ScheduledState.STOPPED); + return null; + } + }).when(mockProcessGroup).stopProcessor(Mockito.any(ProcessorNode.class)); + + final String id1 = UUID.randomUUID().toString(); + final ProcessorNode procNodeA = new StandardProcessorNode(new DummyProcessor(), id1, + new StandardValidationContextFactory(provider), scheduler, provider); + procNodeA.getProcessor().initialize(new StandardProcessorInitializationContext(id1, null, provider)); + procNodeA.setProperty(DummyProcessor.SERVICE.getName(), "1"); + procNodeA.setProcessGroup(mockProcessGroup); + + final String id2 = UUID.randomUUID().toString(); + final ProcessorNode procNodeB = new StandardProcessorNode(new DummyProcessor(),id2, + new StandardValidationContextFactory(provider), scheduler, provider); + procNodeB.getProcessor().initialize(new StandardProcessorInitializationContext(id2, null, provider)); + procNodeB.setProperty(DummyProcessor.SERVICE.getName(), "3"); + procNodeB.setProcessGroup(mockProcessGroup); + + serviceNode1.setProperty(ServiceA.OTHER_SERVICE.getName(), "2"); + serviceNode2.setProperty(ServiceA.OTHER_SERVICE.getName(), "4"); + serviceNode3.setProperty(ServiceA.OTHER_SERVICE.getName(), "2"); + serviceNode3.setProperty(ServiceA.OTHER_SERVICE_2.getName(), "4"); + + provider.enableControllerService(serviceNode4); + provider.enableReferencingServices(serviceNode4); + provider.scheduleReferencingComponents(serviceNode4); + + assertEquals(ControllerServiceState.ENABLED, serviceNode3.getState()); + assertEquals(ControllerServiceState.ENABLED, serviceNode2.getState()); + assertEquals(ControllerServiceState.ENABLED, serviceNode1.getState()); + assertTrue(procNodeA.isRunning()); + assertTrue(procNodeB.isRunning()); + + // stop processors and verify results. + provider.unscheduleReferencingComponents(serviceNode4); + assertFalse(procNodeA.isRunning()); + assertFalse(procNodeB.isRunning()); + assertEquals(ControllerServiceState.ENABLED, serviceNode3.getState()); + assertEquals(ControllerServiceState.ENABLED, serviceNode2.getState()); + assertEquals(ControllerServiceState.ENABLED, serviceNode1.getState()); + + provider.disableReferencingServices(serviceNode4); + assertEquals(ControllerServiceState.DISABLED, serviceNode3.getState()); + assertEquals(ControllerServiceState.DISABLED, serviceNode2.getState()); + assertEquals(ControllerServiceState.DISABLED, serviceNode1.getState()); + assertEquals(ControllerServiceState.ENABLED, serviceNode4.getState()); + + provider.disableControllerService(serviceNode4); + assertEquals(ControllerServiceState.DISABLED, serviceNode4.getState()); + } + + + @Test + public void testOrderingOfServices() { + final StandardControllerServiceProvider provider = new StandardControllerServiceProvider(null, null); + final ControllerServiceNode serviceNode1 = provider.createControllerService(ServiceA.class.getName(), "1", false); + final ControllerServiceNode serviceNode2 = provider.createControllerService(ServiceB.class.getName(), "2", false); + + serviceNode1.setProperty(ServiceA.OTHER_SERVICE.getName(), "2"); + + final Map nodeMap = new LinkedHashMap<>(); + nodeMap.put("1", serviceNode1); + nodeMap.put("2", serviceNode2); + + List> branches = StandardControllerServiceProvider.determineEnablingOrder(nodeMap); + assertEquals(2, branches.size()); + List ordered = branches.get(0); + assertEquals(2, ordered.size()); + assertTrue(ordered.get(0) == serviceNode2); + assertTrue(ordered.get(1) == serviceNode1); + assertEquals(1, branches.get(1).size()); + assertTrue(branches.get(1).get(0) == serviceNode2); + + nodeMap.clear(); + nodeMap.put("2", serviceNode2); + nodeMap.put("1", serviceNode1); + + branches = StandardControllerServiceProvider.determineEnablingOrder(nodeMap); + assertEquals(2, branches.size()); + ordered = branches.get(1); + assertEquals(2, ordered.size()); + assertTrue(ordered.get(0) == serviceNode2); + assertTrue(ordered.get(1) == serviceNode1); + assertEquals(1, branches.get(0).size()); + assertTrue(branches.get(0).get(0) == serviceNode2); + + // add circular dependency on self. + nodeMap.clear(); + serviceNode1.setProperty(ServiceA.OTHER_SERVICE_2.getName(), "1"); + nodeMap.put("1", serviceNode1); + nodeMap.put("2", serviceNode2); + + branches = StandardControllerServiceProvider.determineEnablingOrder(nodeMap); + assertEquals(2, branches.size()); + ordered = branches.get(0); + assertEquals(2, ordered.size()); + assertTrue(ordered.get(0) == serviceNode2); + assertTrue(ordered.get(1) == serviceNode1); + + nodeMap.clear(); + nodeMap.put("2", serviceNode2); + nodeMap.put("1", serviceNode1); + branches = StandardControllerServiceProvider.determineEnablingOrder(nodeMap); + assertEquals(2, branches.size()); + ordered = branches.get(1); + assertEquals(2, ordered.size()); + assertTrue(ordered.get(0) == serviceNode2); + assertTrue(ordered.get(1) == serviceNode1); + + // add circular dependency once removed. In this case, we won't actually be able to enable these because of the + // circular dependency because they will never be valid because they will always depend on a disabled service. + // But we want to ensure that the method returns successfully without throwing a StackOverflowException or anything + // like that. + nodeMap.clear(); + final ControllerServiceNode serviceNode3 = provider.createControllerService(ServiceA.class.getName(), "3", false); + serviceNode1.setProperty(ServiceA.OTHER_SERVICE.getName(), "3"); + serviceNode3.setProperty(ServiceA.OTHER_SERVICE.getName(), "1"); + nodeMap.put("1", serviceNode1); + nodeMap.put("3", serviceNode3); + branches = StandardControllerServiceProvider.determineEnablingOrder(nodeMap); + assertEquals(2, branches.size()); + ordered = branches.get(0); + assertEquals(2, ordered.size()); + assertTrue(ordered.get(0) == serviceNode3); + assertTrue(ordered.get(1) == serviceNode1); + + nodeMap.clear(); + nodeMap.put("3", serviceNode3); + nodeMap.put("1", serviceNode1); + branches = StandardControllerServiceProvider.determineEnablingOrder(nodeMap); + assertEquals(2, branches.size()); + ordered = branches.get(1); + assertEquals(2, ordered.size()); + assertTrue(ordered.get(0) == serviceNode3); + assertTrue(ordered.get(1) == serviceNode1); + + + // Add multiple completely disparate branches. + nodeMap.clear(); + serviceNode1.setProperty(ServiceA.OTHER_SERVICE.getName(), "2"); + final ControllerServiceNode serviceNode4 = provider.createControllerService(ServiceB.class.getName(), "4", false); + final ControllerServiceNode serviceNode5 = provider.createControllerService(ServiceB.class.getName(), "5", false); + serviceNode3.setProperty(ServiceA.OTHER_SERVICE.getName(), "4"); + nodeMap.put("1", serviceNode1); + nodeMap.put("2", serviceNode2); + nodeMap.put("3", serviceNode3); + nodeMap.put("4", serviceNode4); + nodeMap.put("5", serviceNode5); + + branches = StandardControllerServiceProvider.determineEnablingOrder(nodeMap); + assertEquals(5, branches.size()); + + ordered = branches.get(0); + assertEquals(2, ordered.size()); + assertTrue(ordered.get(0) == serviceNode2); + assertTrue(ordered.get(1) == serviceNode1); + + assertEquals(1, branches.get(1).size()); + assertTrue(branches.get(1).get(0) == serviceNode2); + + ordered = branches.get(2); + assertEquals(2, ordered.size()); + assertTrue(ordered.get(0) == serviceNode4); + assertTrue(ordered.get(1) == serviceNode3); + + assertEquals(1, branches.get(3).size()); + assertTrue(branches.get(3).get(0) == serviceNode4); + + assertEquals(1, branches.get(4).size()); + assertTrue(branches.get(4).get(0) == serviceNode5); + + // create 2 branches both dependent on the same service + nodeMap.clear(); + serviceNode1.setProperty(ServiceA.OTHER_SERVICE.getName(), "2"); + serviceNode3.setProperty(ServiceA.OTHER_SERVICE.getName(), "2"); + nodeMap.put("1", serviceNode1); + nodeMap.put("2", serviceNode2); + nodeMap.put("3", serviceNode3); + + branches = StandardControllerServiceProvider.determineEnablingOrder(nodeMap); + assertEquals(3, branches.size()); + + ordered = branches.get(0); + assertEquals(2, ordered.size()); + assertTrue(ordered.get(0) == serviceNode2); + assertTrue(ordered.get(1) == serviceNode1); + + ordered = branches.get(1); + assertEquals(1, ordered.size()); + assertTrue(ordered.get(0) == serviceNode2); + + ordered = branches.get(2); + assertEquals(2, ordered.size()); + assertTrue(ordered.get(0) == serviceNode2); + assertTrue(ordered.get(1) == serviceNode3); + } + +} diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/mock/DummyProcessor.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/mock/DummyProcessor.java new file mode 100644 index 0000000000..615e172ebc --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/mock/DummyProcessor.java @@ -0,0 +1,49 @@ +/* + * 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.controller.service.mock; + +import java.util.ArrayList; +import java.util.List; + +import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.controller.ControllerService; +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 DummyProcessor extends AbstractProcessor { + + public static final PropertyDescriptor SERVICE = new PropertyDescriptor.Builder() + .name("Controller Service") + .identifiesControllerService(ControllerService.class) + .required(true) + .build(); + + + @Override + protected List getSupportedPropertyDescriptors() { + final List descriptors = new ArrayList<>(); + descriptors.add(SERVICE); + return descriptors; + } + + @Override + public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException { + } + +} diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/mock/ServiceA.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/mock/ServiceA.java new file mode 100644 index 0000000000..4918468ea7 --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/mock/ServiceA.java @@ -0,0 +1,49 @@ +/* + * 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.controller.service.mock; + +import java.util.ArrayList; +import java.util.List; + +import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.controller.AbstractControllerService; +import org.apache.nifi.controller.ControllerService; + +public class ServiceA extends AbstractControllerService { + + public static final PropertyDescriptor OTHER_SERVICE = new PropertyDescriptor.Builder() + .name("Other Service") + .identifiesControllerService(ControllerService.class) + .required(true) + .build(); + + public static final PropertyDescriptor OTHER_SERVICE_2 = new PropertyDescriptor.Builder() + .name("Other Service 2") + .identifiesControllerService(ControllerService.class) + .required(false) + .build(); + + + @Override + protected List getSupportedPropertyDescriptors() { + final List descriptors = new ArrayList<>(); + descriptors.add(OTHER_SERVICE); + descriptors.add(OTHER_SERVICE_2); + return descriptors; + } + +} diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/Availability.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/mock/ServiceB.java similarity index 80% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/Availability.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/mock/ServiceB.java index 38df6f754e..070b156e3a 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/Availability.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/mock/ServiceB.java @@ -1,24 +1,23 @@ -/* - * 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.controller; - -public enum Availability { - - CLUSTER_MANAGER_ONLY, - NODE_ONLY, - BOTH; -} +/* + * 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.controller.service.mock; + +import org.apache.nifi.controller.AbstractControllerService; + +public class ServiceB extends AbstractControllerService { + +} diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/processor/TestStandardPropertyValue.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/processor/TestStandardPropertyValue.java index 7390098dac..a0bf30d79c 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/processor/TestStandardPropertyValue.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/processor/TestStandardPropertyValue.java @@ -17,6 +17,7 @@ package org.apache.nifi.processor; import org.apache.nifi.processor.StandardPropertyValue; + import static org.junit.Assert.assertEquals; import java.util.Calendar; @@ -29,7 +30,6 @@ import org.apache.nifi.controller.ControllerService; import org.apache.nifi.controller.ControllerServiceLookup; import org.apache.nifi.controller.repository.StandardFlowFileRecord; import org.apache.nifi.flowfile.FlowFile; - import org.junit.Test; public class TestStandardPropertyValue { @@ -163,5 +163,14 @@ public class TestStandardPropertyValue { return true; } + @Override + public String getControllerServiceName(String serviceIdentifier) { + return null; + } + + @Override + public boolean isControllerServiceEnabling(String serviceIdentifier) { + return false; + } } } diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/.gitignore b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/.gitignore index ea8c4bf7f3..29546b567b 100755 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/.gitignore +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/.gitignore @@ -1 +1,2 @@ /target +/target/ diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/controller-services.xml b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/controller-services.xml deleted file mode 100644 index f5bd96a087..0000000000 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/controller-services.xml +++ /dev/null @@ -1,18 +0,0 @@ - - - - - \ No newline at end of file diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/nifi.properties b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/nifi.properties index fd16cb5363..90b3cdd07f 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/nifi.properties +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/nifi.properties @@ -25,8 +25,6 @@ nifi.administrative.yield.duration=${nifi.administrative.yield.duration} nifi.bored.yield.duration=${nifi.bored.yield.duration} nifi.authority.provider.configuration.file=${nifi.authority.provider.configuration.file} -nifi.reporting.task.configuration.file=${nifi.reporting.task.configuration.file} -nifi.controller.service.configuration.file=${nifi.controller.service.configuration.file} nifi.templates.directory=${nifi.templates.directory} nifi.ui.banner.text=${nifi.ui.banner.text} nifi.ui.autorefresh.interval=${nifi.ui.autorefresh.interval} diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/reporting-tasks.xml b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/reporting-tasks.xml deleted file mode 100644 index 3f60b937ec..0000000000 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/reporting-tasks.xml +++ /dev/null @@ -1,49 +0,0 @@ - - - - NiFi - - status-report - Controller Status Report - org.apache.nifi.controller.ControllerStatusReportingTask - 3 mins - - - - - - - - \ No newline at end of file diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-user-actions/src/main/java/org/apache/nifi/action/Component.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-user-actions/src/main/java/org/apache/nifi/action/Component.java index 62dd68f6ac..7f62c92bd9 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-user-actions/src/main/java/org/apache/nifi/action/Component.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-user-actions/src/main/java/org/apache/nifi/action/Component.java @@ -27,5 +27,7 @@ public enum Component { ProcessGroup, RemoteProcessGroup, Funnel, - Connection; + Connection, + ControllerService, + ReportingTask; } diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-user-actions/src/main/java/org/apache/nifi/action/component/details/ProcessorDetails.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-user-actions/src/main/java/org/apache/nifi/action/component/details/ExtensionDetails.java similarity index 94% rename from nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-user-actions/src/main/java/org/apache/nifi/action/component/details/ProcessorDetails.java rename to nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-user-actions/src/main/java/org/apache/nifi/action/component/details/ExtensionDetails.java index ef0af7c29f..e557548e84 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-user-actions/src/main/java/org/apache/nifi/action/component/details/ProcessorDetails.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-user-actions/src/main/java/org/apache/nifi/action/component/details/ExtensionDetails.java @@ -19,7 +19,7 @@ package org.apache.nifi.action.component.details; /** * */ -public class ProcessorDetails extends ComponentDetails { +public class ExtensionDetails extends ComponentDetails { private String type; diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-custom-ui-utilities/src/main/java/org/apache/nifi/web/HttpServletConfigurationRequestContext.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-custom-ui-utilities/src/main/java/org/apache/nifi/web/HttpServletConfigurationRequestContext.java new file mode 100644 index 0000000000..986ce4cbc4 --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-custom-ui-utilities/src/main/java/org/apache/nifi/web/HttpServletConfigurationRequestContext.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.web; + +import javax.servlet.http.HttpServletRequest; + +/** + * An implementation of the ConfigurationRequestContext that retrieves configuration + * from a HttpServletRequest instance. + */ +public class HttpServletConfigurationRequestContext extends HttpServletRequestContext implements NiFiWebConfigurationRequestContext { + + private static final String CLIENT_ID_PARAM = "clientId"; + private static final String REVISION_PARAM = "revision"; + + private final HttpServletRequest request; + + public HttpServletConfigurationRequestContext(final UiExtensionType extensionType, final HttpServletRequest request) { + super(extensionType, request); + this.request = request; + } + + /** + * @return the revision retrieved from the request parameters with keys + * equal to "clientId" and "revision". + */ + @Override + public Revision getRevision() { + final String revisionParamVal = request.getParameter(REVISION_PARAM); + Long revision; + try { + revision = Long.parseLong(revisionParamVal); + } catch (final Exception ex) { + revision = null; + } + + final String clientId = request.getParameter(CLIENT_ID_PARAM); + + return new Revision(revision, clientId); + } + +} diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-custom-ui-utilities/src/main/java/org/apache/nifi/web/HttpServletRequestContext.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-custom-ui-utilities/src/main/java/org/apache/nifi/web/HttpServletRequestContext.java new file mode 100644 index 0000000000..311fbc7fa4 --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-custom-ui-utilities/src/main/java/org/apache/nifi/web/HttpServletRequestContext.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.web; + +import java.security.cert.X509Certificate; +import javax.servlet.http.HttpServletRequest; + +/** + * An implementation of the ConfigurationRequestContext that retrieves configuration + * from a HttpServletRequest instance. + */ +public class HttpServletRequestContext implements NiFiWebRequestContext { + + private static final String ID_PARAM = "id"; + + private final UiExtensionType extensionType; + private final HttpServletRequest request; + + public HttpServletRequestContext(final UiExtensionType extensionType, final HttpServletRequest request) { + this.extensionType = extensionType; + this.request = request; + } + + @Override + public UiExtensionType getExtensionType() { + return extensionType; + } + + @Override + public String getProxiedEntitiesChain() { + String xProxiedEntitiesChain = request.getHeader("X-ProxiedEntitiesChain"); + final X509Certificate cert = extractClientCertificate(request); + if (cert != null) { + final String extractedPrincipal = extractPrincipal(cert); + final String formattedPrincipal = formatProxyDn(extractedPrincipal); + if (xProxiedEntitiesChain == null || xProxiedEntitiesChain.trim().isEmpty()) { + xProxiedEntitiesChain = formattedPrincipal; + } else { + xProxiedEntitiesChain += formattedPrincipal; + } + } + + return xProxiedEntitiesChain; + } + + /** + * @return the protocol scheme of the HttpServletRequest instance. + */ + @Override + public String getScheme() { + return request.getScheme(); + } + + /** + * @return the ID retrieved from the request parameter with key + * equal to "id". + */ + @Override + public String getId() { + return request.getParameter(ID_PARAM); + } + + /** + * Utility methods that have been copied into this class to reduce the + * dependency footprint of this artifact. These utility methods typically + * live in web-utilities but that would pull in spring, jersey, jackson, + * etc. + */ + private X509Certificate extractClientCertificate(HttpServletRequest request) { + X509Certificate[] certs = (X509Certificate[]) request.getAttribute("javax.servlet.request.X509Certificate"); + + if (certs != null && certs.length > 0) { + return certs[0]; + } + + return null; + } + + private String extractPrincipal(X509Certificate cert) { + return cert.getSubjectDN().getName().trim(); + } + + private String formatProxyDn(String dn) { + return "<" + dn + ">"; + } +} diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-custom-ui-utilities/src/main/java/org/apache/nifi/web/HttpServletRequestContextConfig.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-custom-ui-utilities/src/main/java/org/apache/nifi/web/HttpServletRequestContextConfig.java index e39ebccfb5..e376ab62b7 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-custom-ui-utilities/src/main/java/org/apache/nifi/web/HttpServletRequestContextConfig.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-custom-ui-utilities/src/main/java/org/apache/nifi/web/HttpServletRequestContextConfig.java @@ -23,6 +23,7 @@ import javax.servlet.http.HttpServletRequest; * An implementation of the NiFiWebContextConfig that retrieves configuration * from a HttpServletRequest instance. */ +@Deprecated public class HttpServletRequestContextConfig implements NiFiWebContextConfig { public static final String PROCESSOR_ID_PARAM = "processorId"; 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 9f1cf99dee..4db963753e 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 @@ -123,6 +123,11 @@ nifi-custom-ui-utilities compile + + org.apache.nifi + nifi-ui-extension + compile + org.apache.nifi nifi-web-security diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-jetty/src/main/java/org/apache/nifi/web/server/JettyServer.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-jetty/src/main/java/org/apache/nifi/web/server/JettyServer.java index 54111a1df5..1134c774e3 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-jetty/src/main/java/org/apache/nifi/web/server/JettyServer.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-jetty/src/main/java/org/apache/nifi/web/server/JettyServer.java @@ -53,6 +53,10 @@ import org.apache.nifi.web.NiFiWebContext; import org.apache.commons.collections4.CollectionUtils; import org.apache.commons.lang3.StringUtils; import org.apache.nifi.web.ContentAccess; +import org.apache.nifi.ui.extension.UiExtension; +import org.apache.nifi.ui.extension.UiExtensionMapping; +import org.apache.nifi.web.NiFiWebConfigurationContext; +import org.apache.nifi.web.UiExtensionType; import org.eclipse.jetty.server.Connector; import org.eclipse.jetty.server.Handler; import org.eclipse.jetty.server.HttpConfiguration; @@ -97,14 +101,23 @@ public class JettyServer implements NiFiServer { }; private final Server server; + private final NiFiProperties props; + private ExtensionMapping extensionMapping; private WebAppContext webApiContext; private WebAppContext webDocsContext; + + // content viewer and mime type specific extensions private WebAppContext webContentViewerContext; - private Collection customUiWebContexts; private Collection contentViewerWebContexts; - private final NiFiProperties props; + // component (processor, controller service, reporting task) ui extensions + private UiExtensionMapping componentUiExtensions; + private Collection componentUiExtensionWebContexts; + + @Deprecated + private Collection customUiWebContexts; + /** * Creates and configures a new Jetty instance. * @@ -200,23 +213,34 @@ public class JettyServer implements NiFiServer { // handlers for each war and init params for the web api final HandlerCollection handlers = new HandlerCollection(); - final Map customUiMappings = new HashMap<>(); - final Map mimeTypeMappings = new HashMap<>(); + final Map mimeMappings = new HashMap<>(); final ClassLoader frameworkClassLoader = getClass().getClassLoader(); final ClassLoader jettyClassLoader = frameworkClassLoader.getParent(); + @Deprecated + final Map customUiMappings = new HashMap<>(); + // deploy the other wars if (CollectionUtils.isNotEmpty(otherWars)) { + // hold onto to the web contexts for all ui extensions customUiWebContexts = new ArrayList<>(); + componentUiExtensionWebContexts = new ArrayList<>(); contentViewerWebContexts = new ArrayList<>(); + // ui extension organized by component type + final Map> componentUiExtensionsByType = new HashMap<>(); for (File war : otherWars) { // see if this war is a custom processor ui + @Deprecated List customUiProcessorTypes = getWarExtensions(war, "META-INF/nifi-processor"); - List contentViewerMimeTypes = getWarExtensions(war, "META-INF/nifi-content-viewer"); - // only include wars that are for extensions - if (!customUiProcessorTypes.isEmpty() || !contentViewerMimeTypes.isEmpty()) { + // identify all known extension types in the war + final Map> uiExtensionInWar = new HashMap<>(); + identifyUiExtensionsForComponents(uiExtensionInWar, war); + + // only include wars that are for custom processor ui's + if (!customUiProcessorTypes.isEmpty() || !uiExtensionInWar.isEmpty()) { + // get the context path String warName = StringUtils.substringBeforeLast(war.getName(), "."); String warContextPath = String.format("/%s", warName); @@ -234,24 +258,62 @@ public class JettyServer implements NiFiServer { // also store it by type so we can populate the appropriate initialization parameters if (!customUiProcessorTypes.isEmpty()) { customUiWebContexts.add(extensionUiContext); + + // @Deprecated - supported custom uis as init params to the web api + for (String customUiProcessorType : customUiProcessorTypes) { + // map the processor type to the custom ui path + customUiMappings.put(customUiProcessorType, warContextPath); + } } else { - // record the mime type to web app mapping (need to handle type collision) - contentViewerWebContexts.add(extensionUiContext); + // create the ui extensions + for (final Map.Entry> entry : uiExtensionInWar.entrySet()) { + final UiExtensionType extensionType = entry.getKey(); + final List types = entry.getValue(); + + if (UiExtensionType.ContentViewer.equals(extensionType)) { + // consider each content type identified + for (final String contentType : types) { + // map the content type to the context path + mimeMappings.put(contentType, warContextPath); + } + + // this ui extension provides a content viewer + contentViewerWebContexts.add(extensionUiContext); + } else { + // consider each component type identified + for (final String componentType : types) { + logger.info(String.format("Loading UI extension [%s, %s] for %s", extensionType, warContextPath, types)); + + // record the extension definition + final UiExtension uiExtension = new UiExtension(extensionType, warContextPath); + + // create if this is the first extension for this component type + List componentUiExtensionsForType = componentUiExtensionsByType.get(componentType); + if (componentUiExtensionsForType == null) { + componentUiExtensionsForType = new ArrayList<>(); + componentUiExtensionsByType.put(componentType, componentUiExtensionsForType); + } + + // record this extension + componentUiExtensionsForType.add(uiExtension); + } + + // this ui extension provides a component custom ui + componentUiExtensionWebContexts.add(extensionUiContext); + } + } } // include custom ui web context in the handlers handlers.addHandler(extensionUiContext); - - // add the initialization paramters - for (String customUiProcessorType : customUiProcessorTypes) { - // map the processor type to the custom ui path - customUiMappings.put(customUiProcessorType, warContextPath); - } - for (final String contentViewerMimeType : contentViewerMimeTypes) { - mimeTypeMappings.put(contentViewerMimeType, warContextPath); - } } + } + + // record all ui extensions to give to the web api + componentUiExtensions = new UiExtensionMapping(componentUiExtensionsByType); + } else { + componentUiExtensions = new UiExtensionMapping(Collections.EMPTY_MAP); } // load the web ui app @@ -264,7 +326,7 @@ public class JettyServer implements NiFiServer { // load the content viewer app webContentViewerContext = loadWar(webContentViewerWar, "/nifi-content-viewer", frameworkClassLoader); - webContentViewerContext.getInitParams().putAll(mimeTypeMappings); + webContentViewerContext.getInitParams().putAll(mimeMappings); handlers.addHandler(webContentViewerContext); // create a web app for the docs @@ -315,6 +377,67 @@ public class JettyServer implements NiFiServer { return wars; } + private void readUiExtensions(final Map> uiExtensions, final UiExtensionType uiExtensionType, final JarFile jarFile, final JarEntry jarEntry) throws IOException { + if (jarEntry == null) { + return; + } + + // get an input stream for the nifi-processor configuration file + BufferedReader in = new BufferedReader(new InputStreamReader(jarFile.getInputStream(jarEntry))); + + // read in each configured type + String rawComponentType; + while ((rawComponentType = in.readLine()) != null) { + // extract the component type + final String componentType = extractComponentType(rawComponentType); + if (componentType != null) { + List extensions = uiExtensions.get(uiExtensionType); + + // if there are currently no extensions for this type create it + if (extensions == null) { + extensions = new ArrayList<>(); + uiExtensions.put(uiExtensionType, extensions); + } + + // add the specified type + extensions.add(componentType); + } + } + } + + /** + * Identifies all known UI extensions and stores them in the specified map. + * + * @param uiExtensions + * @param warFile + */ + private void identifyUiExtensionsForComponents(final Map> uiExtensions, final File warFile) { + try (final JarFile jarFile = new JarFile(warFile)) { + // locate the ui extensions + readUiExtensions(uiExtensions, UiExtensionType.ContentViewer, jarFile, jarFile.getJarEntry("META-INF/nifi-content-viewer")); + readUiExtensions(uiExtensions, UiExtensionType.ProcessorConfiguration, jarFile, jarFile.getJarEntry("META-INF/nifi-processor-configuration")); + readUiExtensions(uiExtensions, UiExtensionType.ControllerServiceConfiguration, jarFile, jarFile.getJarEntry("META-INF/nifi-controller-service-configuration")); + readUiExtensions(uiExtensions, UiExtensionType.ReportingTaskConfiguration, jarFile, jarFile.getJarEntry("META-INF/nifi-reporting-task-configuration")); + } catch (IOException ioe) { + logger.warn(String.format("Unable to inspect %s for a UI extensions.", warFile)); + } + } + + /** + * Extracts the component type. Trims the line and considers comments. Returns null if no type was found. + * + * @param line + * @return + */ + private String extractComponentType(final String line) { + final String trimmedLine = line.trim(); + if (!trimmedLine.isEmpty() && !trimmedLine.startsWith("#")) { + final int indexOfPound = trimmedLine.indexOf("#"); + return (indexOfPound > 0) ? trimmedLine.substring(0, indexOfPound) : trimmedLine; + } + return null; + } + /** * Returns the extension in the specified WAR using the specified path. * @@ -335,10 +458,11 @@ public class JettyServer implements NiFiServer { BufferedReader in = new BufferedReader(new InputStreamReader(jarFile.getInputStream(jarEntry))); // read in each configured type - String processorType; - while ((processorType = in.readLine()) != null) { - // ensure the line isn't blank - if (StringUtils.isNotBlank(processorType)) { + String rawProcessorType; + while ((rawProcessorType = in.readLine()) != null) { + // extract the processor type + final String processorType = extractComponentType(rawProcessorType); + if (processorType != null) { processorTypes.add(processorType); } } @@ -558,13 +682,18 @@ public class JettyServer implements NiFiServer { } } } - + // ensure the appropriate wars deployed successfully before injecting the NiFi context and security filters - // this must be done after starting the server (and ensuring there were no start up failures) if (webApiContext != null) { + // give the web api the component ui extensions final ServletContext webApiServletContext = webApiContext.getServletHandler().getServletContext(); + webApiServletContext.setAttribute("nifi-ui-extensions", componentUiExtensions); + + // get the application context final WebApplicationContext webApplicationContext = WebApplicationContextUtils.getRequiredWebApplicationContext(webApiServletContext); + // @Deprecated if (CollectionUtils.isNotEmpty(customUiWebContexts)) { final NiFiWebContext niFiWebContext = webApplicationContext.getBean("nifiWebContext", NiFiWebContext.class); @@ -576,22 +705,40 @@ public class JettyServer implements NiFiServer { // add the security filter to any custom ui wars final FilterHolder securityFilter = webApiContext.getServletHandler().getFilter("springSecurityFilterChain"); if (securityFilter != null) { - customUiContext.addFilter(securityFilter, "/*", EnumSet.of(DispatcherType.REQUEST)); + customUiContext.addFilter(securityFilter, "/*", EnumSet.allOf(DispatcherType.class)); } } } + // component ui extensions + if (CollectionUtils.isNotEmpty(componentUiExtensionWebContexts)) { + final NiFiWebConfigurationContext configurationContext = webApplicationContext.getBean("nifiWebConfigurationContext", NiFiWebConfigurationContext.class); + + for (final WebAppContext customUiContext : componentUiExtensionWebContexts) { + // set the NiFi context in each custom ui servlet context + final ServletContext customUiServletContext = customUiContext.getServletHandler().getServletContext(); + customUiServletContext.setAttribute("nifi-web-configuration-context", configurationContext); + + // add the security filter to any ui extensions wars + final FilterHolder securityFilter = webApiContext.getServletHandler().getFilter("springSecurityFilterChain"); + if (securityFilter != null) { + customUiContext.addFilter(securityFilter, "/*", EnumSet.allOf(DispatcherType.class)); + } + } + } + + // content viewer extensions if (CollectionUtils.isNotEmpty(contentViewerWebContexts)) { for (final WebAppContext contentViewerContext : contentViewerWebContexts) { // add the security filter to any content viewer wars final FilterHolder securityFilter = webApiContext.getServletHandler().getFilter("springSecurityFilterChain"); if (securityFilter != null) { - contentViewerContext.addFilter(securityFilter, "/*", EnumSet.of(DispatcherType.REQUEST, DispatcherType.FORWARD, DispatcherType.INCLUDE)); + contentViewerContext.addFilter(securityFilter, "/*", EnumSet.allOf(DispatcherType.class)); } } } - // ensure the web content viewer war was loaded + // content viewer controller if (webContentViewerContext != null) { final ContentAccess contentAccess = webApplicationContext.getBean("contentAccess", ContentAccess.class); @@ -599,14 +746,13 @@ public class JettyServer implements NiFiServer { final ServletContext webContentViewerServletContext = webContentViewerContext.getServletHandler().getServletContext(); webContentViewerServletContext.setAttribute("nifi-content-access", contentAccess); - // add the security filter to the content viewer controller final FilterHolder securityFilter = webApiContext.getServletHandler().getFilter("springSecurityFilterChain"); if (securityFilter != null) { - webContentViewerContext.addFilter(securityFilter, "/*", EnumSet.of(DispatcherType.REQUEST, DispatcherType.FORWARD, DispatcherType.INCLUDE)); + webContentViewerContext.addFilter(securityFilter, "/*", EnumSet.allOf(DispatcherType.class)); } } } - + // ensure the web document war was loaded and provide the extension mapping if (webDocsContext != null) { final ServletContext webDocsServletContext = webDocsContext.getServletHandler().getServletContext(); @@ -735,4 +881,4 @@ public class JettyServer implements NiFiServer { logger.warn("Failed to stop web server", ex); } } -} +} \ No newline at end of file diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-ui-extension/pom.xml b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-ui-extension/pom.xml new file mode 100644 index 0000000000..22388b398e --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-ui-extension/pom.xml @@ -0,0 +1,21 @@ + + + 4.0.0 + + org.apache.nifi + nifi-web + 0.1.0-incubating-SNAPSHOT + + nifi-ui-extension + jar + + + org.apache.nifi + nifi-api + + + org.apache.nifi + nifi-user-actions + + + \ No newline at end of file diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-ui-extension/src/main/java/org/apache/nifi/ui/extension/UiExtension.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-ui-extension/src/main/java/org/apache/nifi/ui/extension/UiExtension.java new file mode 100644 index 0000000000..e5b9b3e1ea --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-ui-extension/src/main/java/org/apache/nifi/ui/extension/UiExtension.java @@ -0,0 +1,52 @@ +/* + * 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.ui.extension; + +import org.apache.nifi.web.UiExtensionType; + +/** + * Information about a UI extension required to be invoked. + */ +public class UiExtension { + + private final UiExtensionType extensionType; + private final String contextPath; + + public UiExtension(final UiExtensionType extensionType, final String contextPath) { + this.extensionType = extensionType; + this.contextPath = contextPath; + } + + /** + * The type of this UI extension. + * + * @return + */ + public UiExtensionType getExtensionType() { + return extensionType; + } + + /** + * The context path of this UI extenion. + * + * @return + */ + public String getContextPath() { + return contextPath; + } + +} diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-ui-extension/src/main/java/org/apache/nifi/ui/extension/UiExtensionMapping.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-ui-extension/src/main/java/org/apache/nifi/ui/extension/UiExtensionMapping.java new file mode 100644 index 0000000000..16bffd0b43 --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-ui-extension/src/main/java/org/apache/nifi/ui/extension/UiExtensionMapping.java @@ -0,0 +1,52 @@ +/* + * 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.ui.extension; + +import java.util.List; +import java.util.Map; + +/** + * Mapping of all discovered UI extensions. + */ +public class UiExtensionMapping { + + private final Map> uiExtensions; + + public UiExtensionMapping(Map> uiExtensions) { + this.uiExtensions = uiExtensions; + } + + /** + * Returns whether there are any UI extensions for the specified component type. + * + * @param type + * @return + */ + public boolean hasUiExtension(final String type) { + return uiExtensions.containsKey(type); + } + + /** + * Gets the listing of all discovered UI extensions for the specified component type. + * @param type + * @return + */ + public List getUiExtension(final String type) { + return uiExtensions.get(type); + } + +} 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 6a51838923..5a87ff8030 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 @@ -157,6 +157,11 @@ nifi-web-optimistic-locking provided + + org.apache.nifi + nifi-ui-extension + provided + org.apache.nifi nifi-administration diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/ControllerServiceAuditor.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/ControllerServiceAuditor.java new file mode 100644 index 0000000000..ea3af14905 --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/ControllerServiceAuditor.java @@ -0,0 +1,475 @@ +/* + * 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.audit; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.Date; +import java.util.HashMap; +import java.util.Map; +import java.util.Set; + +import org.apache.nifi.action.Action; +import org.apache.nifi.action.Component; +import org.apache.nifi.action.Operation; +import org.apache.nifi.action.component.details.ExtensionDetails; +import org.apache.nifi.action.details.ActionDetails; +import org.apache.nifi.action.details.ConfigureDetails; +import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.controller.ConfiguredComponent; +import org.apache.nifi.controller.ProcessorNode; +import org.apache.nifi.controller.ReportingTaskNode; +import org.apache.nifi.controller.ScheduledState; +import org.apache.nifi.web.security.user.NiFiUserUtils; +import org.apache.nifi.user.NiFiUser; +import org.apache.nifi.controller.service.ControllerServiceNode; +import org.apache.nifi.controller.service.ControllerServiceReference; +import org.apache.nifi.controller.service.ControllerServiceState; +import org.apache.nifi.reporting.ReportingTask; +import org.apache.nifi.web.api.dto.ControllerServiceDTO; +import org.apache.nifi.web.dao.ControllerServiceDAO; +import org.aspectj.lang.ProceedingJoinPoint; +import org.aspectj.lang.annotation.Around; +import org.aspectj.lang.annotation.Aspect; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Audits controller service creation/removal and configuration changes. + */ +@Aspect +public class ControllerServiceAuditor extends NiFiAuditor { + + private static final Logger logger = LoggerFactory.getLogger(ControllerServiceAuditor.class); + + private static final String COMMENTS = "Comments"; + private static final String NAME = "Name"; + private static final String ANNOTATION_DATA = "Annotation Data"; + + /** + * Audits the creation of controller service via createControllerService(). + * + * This method only needs to be run 'after returning'. However, in Java 7 + * the order in which these methods are returned from + * Class.getDeclaredMethods (even though there is no order guaranteed) seems + * to differ from Java 6. SpringAOP depends on this ordering to determine + * advice precedence. By normalizing all advice into Around advice we can + * alleviate this issue. + * + * @param proceedingJoinPoint + * @return + * @throws java.lang.Throwable + */ + @Around("within(org.apache.nifi.web.dao.ControllerServiceDAO+) && " + + "execution(org.apache.nifi.controller.service.ControllerServiceNode createControllerService(org.apache.nifi.web.api.dto.ControllerServiceDTO))") + public ControllerServiceNode createControllerServiceAdvice(ProceedingJoinPoint proceedingJoinPoint) throws Throwable { + // update the controller service state + ControllerServiceNode controllerService = (ControllerServiceNode) proceedingJoinPoint.proceed(); + + // if no exceptions were thrown, add the controller service action... + final Action action = generateAuditRecord(controllerService, Operation.Add); + + // save the actions + if (action != null) { + saveAction(action, logger); + } + + return controllerService; + } + + /** + * Audits the configuration of a single controller service. + * + * @param proceedingJoinPoint + * @param controllerServiceDTO + * @param controllerServiceDAO + * @return + * @throws Throwable + */ + @Around("within(org.apache.nifi.web.dao.ControllerServiceDAO+) && " + + "execution(org.apache.nifi.controller.service.ControllerServiceNode updateControllerService(org.apache.nifi.web.api.dto.ControllerServiceDTO)) && " + + "args(controllerServiceDTO) && " + + "target(controllerServiceDAO)") + public Object updateControllerServiceAdvice(ProceedingJoinPoint proceedingJoinPoint, ControllerServiceDTO controllerServiceDTO, ControllerServiceDAO controllerServiceDAO) throws Throwable { + // determine the initial values for each property/setting thats changing + ControllerServiceNode controllerService = controllerServiceDAO.getControllerService(controllerServiceDTO.getId()); + final Map values = extractConfiguredPropertyValues(controllerService, controllerServiceDTO); + final boolean isDisabled = isDisabled(controllerService); + + // update the controller service state + final ControllerServiceNode updatedControllerService = (ControllerServiceNode) proceedingJoinPoint.proceed(); + + // if no exceptions were thrown, add the controller service action... + controllerService = controllerServiceDAO.getControllerService(updatedControllerService.getIdentifier()); + + // get the current user + NiFiUser user = NiFiUserUtils.getNiFiUser(); + + // ensure the user was found + if (user != null) { + // determine the updated values + Map updatedValues = extractConfiguredPropertyValues(controllerService, controllerServiceDTO); + + // create the controller service details + ExtensionDetails serviceDetails = new ExtensionDetails(); + serviceDetails.setType(controllerService.getControllerServiceImplementation().getClass().getSimpleName()); + + // create a controller service action + Date actionTimestamp = new Date(); + Collection actions = new ArrayList<>(); + + // go through each updated value + for (String property : updatedValues.keySet()) { + String newValue = updatedValues.get(property); + String oldValue = values.get(property); + Operation operation = null; + + // determine the type of operation + if (oldValue == null || newValue == null || !newValue.equals(oldValue)) { + operation = Operation.Configure; + } + + // create a configuration action accordingly + if (operation != null) { + // clear the value if this property is sensitive + final PropertyDescriptor propertyDescriptor = controllerService.getControllerServiceImplementation().getPropertyDescriptor(property); + if (propertyDescriptor != null && propertyDescriptor.isSensitive()) { + if (newValue != null) { + newValue = "********"; + } + if (oldValue != null) { + oldValue = "********"; + } + } else if (ANNOTATION_DATA.equals(property)) { + if (newValue != null) { + newValue = ""; + } + if (oldValue != null) { + oldValue = ""; + } + } + + final ConfigureDetails actionDetails = new ConfigureDetails(); + actionDetails.setName(property); + actionDetails.setValue(newValue); + actionDetails.setPreviousValue(oldValue); + + // create a configuration action + Action configurationAction = new Action(); + configurationAction.setUserDn(user.getDn()); + configurationAction.setUserName(user.getUserName()); + configurationAction.setOperation(operation); + configurationAction.setTimestamp(actionTimestamp); + configurationAction.setSourceId(controllerService.getIdentifier()); + configurationAction.setSourceName(controllerService.getName()); + configurationAction.setSourceType(Component.ControllerService); + configurationAction.setComponentDetails(serviceDetails); + configurationAction.setActionDetails(actionDetails); + actions.add(configurationAction); + } + } + + // determine the new executing state + final boolean updateIsDisabled = isDisabled(updatedControllerService); + + // determine if the running state has changed and its not disabled + if (isDisabled != updateIsDisabled) { + // create a controller service action + Action serviceAction = new Action(); + serviceAction.setUserDn(user.getDn()); + serviceAction.setUserName(user.getUserName()); + serviceAction.setTimestamp(new Date()); + serviceAction.setSourceId(controllerService.getIdentifier()); + serviceAction.setSourceName(controllerService.getName()); + serviceAction.setSourceType(Component.ControllerService); + serviceAction.setComponentDetails(serviceDetails); + + // set the operation accordingly + if (updateIsDisabled) { + serviceAction.setOperation(Operation.Disable); + } else { + serviceAction.setOperation(Operation.Enable); + } + actions.add(serviceAction); + } + + // ensure there are actions to record + if (!actions.isEmpty()) { + // save the actions + saveActions(actions, logger); + } + } + + return updatedControllerService; + } + + /** + * Audits the update of a component referencing a controller service. + * + * @param proceedingJoinPoint + * @param controllerServiceId + * @return + * @throws Throwable + */ + @Around("within(org.apache.nifi.web.dao.ControllerServiceDAO+) && " + + "execution(org.apache.nifi.controller.service.ControllerServiceReference updateControllerServiceReferencingComponents(java.lang.String, org.apache.nifi.controller.ScheduledState, org.apache.nifi.controller.service.ControllerServiceState))") + public Object updateControllerServiceReferenceAdvice(ProceedingJoinPoint proceedingJoinPoint) throws Throwable { + // update the controller service references + final ControllerServiceReference controllerServiceReference = (ControllerServiceReference) proceedingJoinPoint.proceed(); + + // get the current user + final NiFiUser user = NiFiUserUtils.getNiFiUser(); + + if (user != null) { + final Collection actions = new ArrayList<>(); + final Collection visitedServices = new ArrayList<>(); + visitedServices.add(controllerServiceReference.getReferencedComponent().getIdentifier()); + + // get all applicable actions + getUpdateActionsForReferencingComponents(user, actions, visitedServices, controllerServiceReference.getReferencingComponents()); + + // ensure there are actions to record + if (!actions.isEmpty()) { + // save the actions + saveActions(actions, logger); + } + } + + return controllerServiceReference; + } + + /** + * Gets the update actions for all specified referencing components. + * + * @param user + * @param actions + * @param visitedServices + * @param referencingComponents + */ + private void getUpdateActionsForReferencingComponents(final NiFiUser user, final Collection actions, final Collection visitedServices, final Set referencingComponents) { + // consider each component updates + for (final ConfiguredComponent component : referencingComponents) { + if (component instanceof ProcessorNode) { + final ProcessorNode processor = ((ProcessorNode) component); + + // create the processor details + ExtensionDetails processorDetails = new ExtensionDetails(); + processorDetails.setType(processor.getProcessor().getClass().getSimpleName()); + + // create a processor action + Action processorAction = new Action(); + processorAction.setUserDn(user.getDn()); + processorAction.setUserName(user.getUserName()); + processorAction.setTimestamp(new Date()); + processorAction.setSourceId(processor.getIdentifier()); + processorAction.setSourceName(processor.getName()); + processorAction.setSourceType(Component.Processor); + processorAction.setComponentDetails(processorDetails); + processorAction.setOperation(ScheduledState.RUNNING.equals(processor.getScheduledState()) ? Operation.Start : Operation.Stop); + actions.add(processorAction); + } else if (component instanceof ReportingTask) { + final ReportingTaskNode reportingTask = ((ReportingTaskNode) component); + + // create the reporting task details + ExtensionDetails processorDetails = new ExtensionDetails(); + processorDetails.setType(reportingTask.getReportingTask().getClass().getSimpleName()); + + // create a reporting task action + Action reportingTaskAction = new Action(); + reportingTaskAction.setUserDn(user.getDn()); + reportingTaskAction.setUserName(user.getUserName()); + reportingTaskAction.setTimestamp(new Date()); + reportingTaskAction.setSourceId(reportingTask.getIdentifier()); + reportingTaskAction.setSourceName(reportingTask.getName()); + reportingTaskAction.setSourceType(Component.ReportingTask); + reportingTaskAction.setComponentDetails(processorDetails); + reportingTaskAction.setOperation(ScheduledState.RUNNING.equals(reportingTask.getScheduledState()) ? Operation.Start : Operation.Stop); + actions.add(reportingTaskAction); + } else if (component instanceof ControllerServiceNode) { + final ControllerServiceNode controllerService = ((ControllerServiceNode) component); + + // create the controller service details + ExtensionDetails serviceDetails = new ExtensionDetails(); + serviceDetails.setType(controllerService.getControllerServiceImplementation().getClass().getSimpleName()); + + // create a controller service action + Action serviceAction = new Action(); + serviceAction.setUserDn(user.getDn()); + serviceAction.setUserName(user.getUserName()); + serviceAction.setTimestamp(new Date()); + serviceAction.setSourceId(controllerService.getIdentifier()); + serviceAction.setSourceName(controllerService.getName()); + serviceAction.setSourceType(Component.ControllerService); + serviceAction.setComponentDetails(serviceDetails); + serviceAction.setOperation(isDisabled(controllerService) ? Operation.Disable : Operation.Enable); + actions.add(serviceAction); + + // need to consider components referencing this controller service (transitive) + if (!visitedServices.contains(controllerService.getIdentifier())) { + getUpdateActionsForReferencingComponents(user, actions, visitedServices, controllerService.getReferences().getReferencingComponents()); + } + } + } + } + + /** + * Audits the removal of a controller service via deleteControllerService(). + * + * @param proceedingJoinPoint + * @param controllerServiceId + * @param controllerServiceDAO + * @throws Throwable + */ + @Around("within(org.apache.nifi.web.dao.ControllerServiceDAO+) && " + + "execution(void deleteControllerService(java.lang.String)) && " + + "args(controllerServiceId) && " + + "target(controllerServiceDAO)") + public void removeControllerServiceAdvice(ProceedingJoinPoint proceedingJoinPoint, String controllerServiceId, ControllerServiceDAO controllerServiceDAO) throws Throwable { + // get the controller service before removing it + ControllerServiceNode controllerService = controllerServiceDAO.getControllerService(controllerServiceId); + + // remove the controller service + proceedingJoinPoint.proceed(); + + // if no exceptions were thrown, add removal actions... + // audit the controller service removal + final Action action = generateAuditRecord(controllerService, Operation.Remove); + + // save the actions + if (action != null) { + saveAction(action, logger); + } + } + + /** + * Generates an audit record for the creation of a controller service. + * + * @param controllerService + * @param operation + * @return + */ + private Action generateAuditRecord(ControllerServiceNode controllerService, Operation operation) { + return generateAuditRecord(controllerService, operation, null); + } + + /** + * Generates an audit record for the creation of a controller service. + * + * @param controllerService + * @param operation + * @param actionDetails + * @return + */ + private Action generateAuditRecord(ControllerServiceNode controllerService, Operation operation, ActionDetails actionDetails) { + Action action = null; + + // get the current user + NiFiUser user = NiFiUserUtils.getNiFiUser(); + + // ensure the user was found + if (user != null) { + // create the controller service details + ExtensionDetails serviceDetails = new ExtensionDetails(); + serviceDetails.setType(controllerService.getControllerServiceImplementation().getClass().getSimpleName()); + + // create the controller service action for adding this controller service + action = new Action(); + action.setUserDn(user.getDn()); + action.setUserName(user.getUserName()); + action.setOperation(operation); + action.setTimestamp(new Date()); + action.setSourceId(controllerService.getIdentifier()); + action.setSourceName(controllerService.getName()); + action.setSourceType(Component.ControllerService); + action.setComponentDetails(serviceDetails); + + if (actionDetails != null) { + action.setActionDetails(actionDetails); + } + } + + return action; + } + + /** + * Extracts the values for the configured properties from the specified + * ControllerService. + * + * @param controllerService + * @param controllerServiceDTO + * @return + */ + private Map extractConfiguredPropertyValues(ControllerServiceNode controllerService, ControllerServiceDTO controllerServiceDTO) { + Map values = new HashMap<>(); + + if (controllerServiceDTO.getName() != null) { + values.put(NAME, controllerService.getName()); + } + if (controllerServiceDTO.getAnnotationData() != null) { + values.put(ANNOTATION_DATA, controllerService.getAnnotationData()); + } + if (controllerServiceDTO.getProperties() != null) { + // for each property specified, extract its configured value + Map properties = controllerServiceDTO.getProperties(); + Map configuredProperties = controllerService.getProperties(); + for (String propertyName : properties.keySet()) { + // build a descriptor for getting the configured value + PropertyDescriptor propertyDescriptor = new PropertyDescriptor.Builder().name(propertyName).build(); + String configuredPropertyValue = configuredProperties.get(propertyDescriptor); + + // if the configured value couldn't be found, use the default value from the actual descriptor + if (configuredPropertyValue == null) { + propertyDescriptor = locatePropertyDescriptor(configuredProperties.keySet(), propertyDescriptor); + configuredPropertyValue = propertyDescriptor.getDefaultValue(); + } + values.put(propertyName, configuredPropertyValue); + } + } + if (controllerServiceDTO.getComments() != null) { + values.put(COMMENTS, controllerService.getComments()); + } + + return values; + } + + /** + * Locates the actual property descriptor for the given spec property + * descriptor. + * + * @param propertyDescriptors + * @param specDescriptor + * @return + */ + private PropertyDescriptor locatePropertyDescriptor(Set propertyDescriptors, PropertyDescriptor specDescriptor) { + for (PropertyDescriptor propertyDescriptor : propertyDescriptors) { + if (propertyDescriptor.equals(specDescriptor)) { + return propertyDescriptor; + } + } + return specDescriptor; + } + + /** + * Returns whether the specified controller service is disabled (or disabling). + * + * @param controllerService + * @return + */ + private boolean isDisabled(final ControllerServiceNode controllerService) { + return ControllerServiceState.DISABLED.equals(controllerService.getState()) || ControllerServiceState.DISABLING.equals(controllerService.getState()); + } +} diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/FunnelAuditor.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/FunnelAuditor.java index ef66dc6975..b079da893b 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/FunnelAuditor.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/FunnelAuditor.java @@ -50,7 +50,7 @@ public class FunnelAuditor extends NiFiAuditor { */ @Around("within(org.apache.nifi.web.dao.FunnelDAO+) && " + "execution(org.apache.nifi.connectable.Funnel createFunnel(java.lang.String, org.apache.nifi.web.api.dto.FunnelDTO))") - public Object createFunnelAdvice(ProceedingJoinPoint proceedingJoinPoint) throws Throwable { + public Funnel createFunnelAdvice(ProceedingJoinPoint proceedingJoinPoint) throws Throwable { // perform the underlying operation Funnel funnel = (Funnel) proceedingJoinPoint.proceed(); @@ -69,6 +69,7 @@ public class FunnelAuditor extends NiFiAuditor { * Audits the removal of a funnel. * * @param proceedingJoinPoint + * @param groupId * @param funnelId * @param funnelDAO * @throws Throwable @@ -97,6 +98,8 @@ public class FunnelAuditor extends NiFiAuditor { * Generates an audit record for the creation of the specified funnel. * * @param funnel + * @param operation + * @return */ public Action generateAuditRecord(Funnel funnel, Operation operation) { return generateAuditRecord(funnel, operation, null); @@ -106,6 +109,9 @@ public class FunnelAuditor extends NiFiAuditor { * Generates an audit record for the creation of the specified funnel. * * @param funnel + * @param operation + * @param actionDetails + * @return */ public Action generateAuditRecord(Funnel funnel, Operation operation, ActionDetails actionDetails) { Action action = null; diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/NiFiAuditor.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/NiFiAuditor.java index 046d5ff7d1..adff9d1d4c 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/NiFiAuditor.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/NiFiAuditor.java @@ -54,14 +54,13 @@ public abstract class NiFiAuditor { * Records the actions. * * @param actions + * @param logger */ protected void saveActions(Collection actions, Logger logger) { - /* - * if we're a clustered node, then set actions on threadlocal - */ - if (serviceFacade.isClustered()) { - // if we're a connected node, then put audit actions on threadlocal to propagate back to manager - ClusterContext ctx = ClusterContextThreadLocal.getContext(); + ClusterContext ctx = ClusterContextThreadLocal.getContext(); + + // if we're a connected node, then put audit actions on threadlocal to propagate back to manager + if (ctx != null) { ctx.getActions().addAll(actions); } else { // if we're the cluster manager, or a disconnected node, or running standalone, then audit actions diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/PortAuditor.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/PortAuditor.java index 58dcc39503..af4b5bd06b 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/PortAuditor.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/PortAuditor.java @@ -60,7 +60,7 @@ public class PortAuditor extends NiFiAuditor { */ @Around("within(org.apache.nifi.web.dao.PortDAO+) && " + "execution(org.apache.nifi.connectable.Port createPort(java.lang.String, org.apache.nifi.web.api.dto.PortDTO))") - public Object createPortAdvice(ProceedingJoinPoint proceedingJoinPoint) throws Throwable { + public Port createPortAdvice(ProceedingJoinPoint proceedingJoinPoint) throws Throwable { // perform the underlying operation Port port = (Port) proceedingJoinPoint.proceed(); @@ -79,6 +79,9 @@ public class PortAuditor extends NiFiAuditor { * Audits the update of a port. * * @param proceedingJoinPoint + * @param groupId + * @param portDTO + * @param portDAO * @return * @throws Throwable */ @@ -86,7 +89,7 @@ public class PortAuditor extends NiFiAuditor { + "execution(org.apache.nifi.connectable.Port updatePort(java.lang.String, org.apache.nifi.web.api.dto.PortDTO)) && " + "args(groupId, portDTO) && " + "target(portDAO)") - public Object updatePortAdvice(ProceedingJoinPoint proceedingJoinPoint, String groupId, PortDTO portDTO, PortDAO portDAO) throws Throwable { + public Port updatePortAdvice(ProceedingJoinPoint proceedingJoinPoint, String groupId, PortDTO portDTO, PortDAO portDAO) throws Throwable { final Port port = portDAO.getPort(groupId, portDTO.getId()); final ScheduledState scheduledState = port.getScheduledState(); final String name = port.getName(); @@ -261,8 +264,9 @@ public class PortAuditor extends NiFiAuditor { * Audits the removal of a processor via deleteProcessor(). * * @param proceedingJoinPoint - * @param processorId - * @param processorDAO + * @param groupId + * @param portId + * @param portDAO * @throws Throwable */ @Around("within(org.apache.nifi.web.dao.PortDAO+) && " @@ -290,6 +294,8 @@ public class PortAuditor extends NiFiAuditor { * Generates an audit record for the creation of the specified port. * * @param port + * @param operation + * @return */ public Action generateAuditRecord(Port port, Operation operation) { return generateAuditRecord(port, operation, null); @@ -299,6 +305,9 @@ public class PortAuditor extends NiFiAuditor { * Generates an audit record for the creation of the specified port. * * @param port + * @param operation + * @param actionDetails + * @return */ public Action generateAuditRecord(Port port, Operation operation, ActionDetails actionDetails) { Action action = null; diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/ProcessGroupAuditor.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/ProcessGroupAuditor.java index 1b2af7d999..43918812cf 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/ProcessGroupAuditor.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/ProcessGroupAuditor.java @@ -55,15 +55,12 @@ public class ProcessGroupAuditor extends NiFiAuditor { * alleviate this issue. * * @param proceedingJoinPoint + * @return + * @throws java.lang.Throwable */ -// @AfterReturning( -// pointcut="within(org.apache.nifi.web.dao.ProcessGroupDAO+) && " -// + "execution(org.apache.nifi.web.api.dto.ProcessGroupDTO createProcessGroup(java.lang.String, org.apache.nifi.web.api.dto.ProcessGroupDTO))", -// returning="processGroup" -// ) @Around("within(org.apache.nifi.web.dao.ProcessGroupDAO+) && " + "execution(org.apache.nifi.groups.ProcessGroup createProcessGroup(java.lang.String, org.apache.nifi.web.api.dto.ProcessGroupDTO))") - public Object createProcessGroupAdvice(ProceedingJoinPoint proceedingJoinPoint) throws Throwable { + public ProcessGroup createProcessGroupAdvice(ProceedingJoinPoint proceedingJoinPoint) throws Throwable { // create the process group ProcessGroup processGroup = (ProcessGroup) proceedingJoinPoint.proceed(); @@ -83,13 +80,14 @@ public class ProcessGroupAuditor extends NiFiAuditor { * Audits the update of process group configuration. * * @param proceedingJoinPoint + * @param processGroupDTO * @return * @throws Throwable */ @Around("within(org.apache.nifi.web.dao.ProcessGroupDAO+) && " + "execution(org.apache.nifi.groups.ProcessGroup updateProcessGroup(org.apache.nifi.web.api.dto.ProcessGroupDTO)) && " + "args(processGroupDTO)") - public Object updateProcessGroupAdvice(ProceedingJoinPoint proceedingJoinPoint, ProcessGroupDTO processGroupDTO) throws Throwable { + public ProcessGroup updateProcessGroupAdvice(ProceedingJoinPoint proceedingJoinPoint, ProcessGroupDTO processGroupDTO) throws Throwable { ProcessGroupDAO processGroupDAO = getProcessGroupDAO(); ProcessGroup processGroup = processGroupDAO.getProcessGroup(processGroupDTO.getId()); @@ -170,7 +168,7 @@ public class ProcessGroupAuditor extends NiFiAuditor { processGroupAction.setTimestamp(new Date()); // determine the running state - if (processGroupDTO.isRunning().booleanValue()) { + if (processGroupDTO.isRunning()) { processGroupAction.setOperation(Operation.Start); } else { processGroupAction.setOperation(Operation.Stop); @@ -194,7 +192,6 @@ public class ProcessGroupAuditor extends NiFiAuditor { * * @param proceedingJoinPoint * @param groupId - * @param processGroupDAO * @throws Throwable */ @Around("within(org.apache.nifi.web.dao.ProcessGroupDAO+) && " @@ -222,6 +219,7 @@ public class ProcessGroupAuditor extends NiFiAuditor { * Generates an audit record for the creation of a process group. * * @param processGroup + * @param operation * @return */ public Action generateAuditRecord(ProcessGroup processGroup, Operation operation) { @@ -232,6 +230,8 @@ public class ProcessGroupAuditor extends NiFiAuditor { * Generates an audit record for the creation of a process group. * * @param processGroup + * @param operation + * @param actionDetails * @return */ public Action generateAuditRecord(ProcessGroup processGroup, Operation operation, ActionDetails actionDetails) { diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/ProcessorAuditor.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/ProcessorAuditor.java index c3c70c4780..ff377a3413 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/ProcessorAuditor.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/ProcessorAuditor.java @@ -30,7 +30,7 @@ import java.util.Set; import org.apache.nifi.action.Action; import org.apache.nifi.action.Component; import org.apache.nifi.action.Operation; -import org.apache.nifi.action.component.details.ProcessorDetails; +import org.apache.nifi.action.component.details.ExtensionDetails; import org.apache.nifi.action.details.ActionDetails; import org.apache.nifi.action.details.ConfigureDetails; import org.apache.nifi.components.PropertyDescriptor; @@ -78,13 +78,10 @@ public class ProcessorAuditor extends NiFiAuditor { * advice precedence. By normalizing all advice into Around advice we can * alleviate this issue. * - * @param processor + * @param proceedingJoinPoint + * @return + * @throws java.lang.Throwable */ -// @AfterReturning( -// pointcut="within(org.apache.nifi.web.dao.ProcessorDAO+) && " -// + "execution(org.apache.nifi.web.api.dto.ProcessorDTO createProcessor(org.apache.nifi.web.api.dto.ProcessorDTO))", -// returning="processor" -// ) @Around("within(org.apache.nifi.web.dao.ProcessorDAO+) && " + "execution(org.apache.nifi.controller.ProcessorNode createProcessor(java.lang.String, org.apache.nifi.web.api.dto.ProcessorDTO))") public ProcessorNode createProcessorAdvice(ProceedingJoinPoint proceedingJoinPoint) throws Throwable { @@ -106,6 +103,7 @@ public class ProcessorAuditor extends NiFiAuditor { * Audits the configuration of a single processor. * * @param proceedingJoinPoint + * @param groupId * @param processorDTO * @param processorDAO * @return @@ -115,7 +113,7 @@ public class ProcessorAuditor extends NiFiAuditor { + "execution(org.apache.nifi.controller.ProcessorNode updateProcessor(java.lang.String, org.apache.nifi.web.api.dto.ProcessorDTO)) && " + "args(groupId, processorDTO) && " + "target(processorDAO)") - public Object updateProcessorAdvice(ProceedingJoinPoint proceedingJoinPoint, String groupId, ProcessorDTO processorDTO, ProcessorDAO processorDAO) throws Throwable { + public ProcessorNode updateProcessorAdvice(ProceedingJoinPoint proceedingJoinPoint, String groupId, ProcessorDTO processorDTO, ProcessorDAO processorDAO) throws Throwable { // determine the initial values for each property/setting thats changing ProcessorNode processor = processorDAO.getProcessor(groupId, processorDTO.getId()); final Map values = extractConfiguredPropertyValues(processor, processorDTO); @@ -137,7 +135,7 @@ public class ProcessorAuditor extends NiFiAuditor { Map updatedValues = extractConfiguredPropertyValues(processor, processorDTO); // create the processor details - ProcessorDetails processorDetails = new ProcessorDetails(); + ExtensionDetails processorDetails = new ExtensionDetails(); processorDetails.setType(processor.getProcessor().getClass().getSimpleName()); // create a processor action @@ -240,6 +238,7 @@ public class ProcessorAuditor extends NiFiAuditor { * Audits the removal of a processor via deleteProcessor(). * * @param proceedingJoinPoint + * @param groupId * @param processorId * @param processorDAO * @throws Throwable @@ -281,6 +280,7 @@ public class ProcessorAuditor extends NiFiAuditor { * * @param processor * @param operation + * @param actionDetails * @return */ public Action generateAuditRecord(ProcessorNode processor, Operation operation, ActionDetails actionDetails) { @@ -292,7 +292,7 @@ public class ProcessorAuditor extends NiFiAuditor { // ensure the user was found if (user != null) { // create the processor details - ProcessorDetails processorDetails = new ProcessorDetails(); + ExtensionDetails processorDetails = new ExtensionDetails(); processorDetails.setType(processor.getProcessor().getClass().getSimpleName()); // create the processor action for adding this processor diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/RelationshipAuditor.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/RelationshipAuditor.java index 9cc694bcf1..ba7814194b 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/RelationshipAuditor.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/RelationshipAuditor.java @@ -76,16 +76,13 @@ public class RelationshipAuditor extends NiFiAuditor { * advice precedence. By normalizing all advice into Around advice we can * alleviate this issue. * - * @param connection + * @param proceedingJoinPoint + * @return + * @throws java.lang.Throwable */ -// @AfterReturning( -// pointcut="within(org.apache.nifi.view.dao.ConnectionDAO+) && " -// + "execution(org.apache.nifi.view.model.Connection createConnection(org.apache.nifi.api.dto.ConnectionDTO))", -// returning="connection" -// ) @Around("within(org.apache.nifi.web.dao.ConnectionDAO+) && " + "execution(org.apache.nifi.connectable.Connection createConnection(java.lang.String, org.apache.nifi.web.api.dto.ConnectionDTO))") - public Object createConnectionAdvice(ProceedingJoinPoint proceedingJoinPoint) throws Throwable { + public Connection createConnectionAdvice(ProceedingJoinPoint proceedingJoinPoint) throws Throwable { // perform the underlying operation Connection connection = (Connection) proceedingJoinPoint.proceed(); @@ -105,6 +102,7 @@ public class RelationshipAuditor extends NiFiAuditor { * Audits the creation and removal of relationships via updateConnection(). * * @param proceedingJoinPoint + * @param groupId * @param connectionDTO * @param connectionDAO * @return @@ -114,7 +112,7 @@ public class RelationshipAuditor extends NiFiAuditor { + "execution(org.apache.nifi.connectable.Connection updateConnection(java.lang.String, org.apache.nifi.web.api.dto.ConnectionDTO)) && " + "args(groupId, connectionDTO) && " + "target(connectionDAO)") - public Object updateConnectionAdvice(ProceedingJoinPoint proceedingJoinPoint, String groupId, ConnectionDTO connectionDTO, ConnectionDAO connectionDAO) throws Throwable { + public Connection updateConnectionAdvice(ProceedingJoinPoint proceedingJoinPoint, String groupId, ConnectionDTO connectionDTO, ConnectionDAO connectionDAO) throws Throwable { // get the previous configuration Connection connection = connectionDAO.getConnection(groupId, connectionDTO.getId()); Connectable previousDestination = connection.getDestination(); @@ -218,6 +216,7 @@ public class RelationshipAuditor extends NiFiAuditor { * Audits the removal of relationships via deleteConnection(). * * @param proceedingJoinPoint + * @param groupId * @param id * @param connectionDAO * @throws Throwable @@ -251,7 +250,9 @@ public class RelationshipAuditor extends NiFiAuditor { * Creates action details for connect/disconnect actions. * * @param connection + * @param source * @param relationships + * @param destination * @return */ public ConnectDetails createConnectDetails(final Connection connection, final Connectable source, final Collection relationships, final Connectable destination) { @@ -327,6 +328,7 @@ public class RelationshipAuditor extends NiFiAuditor { * * @param connection * @param operation + * @param actionDetails * @return */ public Action generateAuditRecordForConnection(Connection connection, Operation operation, ActionDetails actionDetails) { diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/RemoteProcessGroupAuditor.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/RemoteProcessGroupAuditor.java index a10495d6c2..d0836d4adb 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/RemoteProcessGroupAuditor.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/RemoteProcessGroupAuditor.java @@ -60,16 +60,13 @@ public class RemoteProcessGroupAuditor extends NiFiAuditor { * advice precedence. By normalizing all advice into Around advice we can * alleviate this issue. * - * @param processor + * @param proceedingJoinPoint + * @return + * @throws java.lang.Throwable */ -// @AfterReturning( -// pointcut="within(org.apache.nifi.web.dao.RemoteProcessGroupDAO+) && " -// + "execution(org.apache.nifi.web.api.dto.RemoteProcessGroupDTO createRemoteProcessGroup(java.lang.String, org.apache.nifi.web.api.dto.RemoteProcessGroupDTO))", -// returning="remoteProcessGroup" -// ) @Around("within(org.apache.nifi.web.dao.RemoteProcessGroupDAO+) && " + "execution(org.apache.nifi.groups.RemoteProcessGroup createRemoteProcessGroup(java.lang.String, org.apache.nifi.web.api.dto.RemoteProcessGroupDTO))") - public Object createRemoteProcessGroupAdvice(ProceedingJoinPoint proceedingJoinPoint) throws Throwable { + public RemoteProcessGroup createRemoteProcessGroupAdvice(ProceedingJoinPoint proceedingJoinPoint) throws Throwable { // create the remote process group RemoteProcessGroup remoteProcessGroup = (RemoteProcessGroup) proceedingJoinPoint.proceed(); @@ -99,7 +96,7 @@ public class RemoteProcessGroupAuditor extends NiFiAuditor { + "execution(org.apache.nifi.groups.RemoteProcessGroup updateRemoteProcessGroup(java.lang.String, org.apache.nifi.web.api.dto.RemoteProcessGroupDTO)) && " + "args(groupId, remoteProcessGroupDTO) && " + "target(remoteProcessGroupDAO)") - public Object auditUpdateProcessGroupConfiguration(ProceedingJoinPoint proceedingJoinPoint, String groupId, RemoteProcessGroupDTO remoteProcessGroupDTO, RemoteProcessGroupDAO remoteProcessGroupDAO) throws Throwable { + public RemoteProcessGroup auditUpdateProcessGroupConfiguration(ProceedingJoinPoint proceedingJoinPoint, String groupId, RemoteProcessGroupDTO remoteProcessGroupDTO, RemoteProcessGroupDAO remoteProcessGroupDAO) throws Throwable { final RemoteProcessGroup remoteProcessGroup = remoteProcessGroupDAO.getRemoteProcessGroup(groupId, remoteProcessGroupDTO.getId()); // record the current value of this remoteProcessGroups configuration for comparisons later @@ -332,7 +329,9 @@ public class RemoteProcessGroupAuditor extends NiFiAuditor { /** * Generates an audit record for the specified remote process group. * - * @param port + * @param remoteProcessGroup + * @param operation + * @return */ public Action generateAuditRecord(RemoteProcessGroup remoteProcessGroup, Operation operation) { return generateAuditRecord(remoteProcessGroup, operation, null); @@ -341,7 +340,10 @@ public class RemoteProcessGroupAuditor extends NiFiAuditor { /** * Generates an audit record for the specified remote process group. * - * @param port + * @param remoteProcessGroup + * @param operation + * @param actionDetails + * @return */ public Action generateAuditRecord(RemoteProcessGroup remoteProcessGroup, Operation operation, ActionDetails actionDetails) { Action action = null; diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/ReportingTaskAuditor.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/ReportingTaskAuditor.java new file mode 100644 index 0000000000..ba2cdfb4c5 --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/ReportingTaskAuditor.java @@ -0,0 +1,353 @@ +/* + * 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.audit; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.Date; +import java.util.HashMap; +import java.util.Map; +import java.util.Set; + +import org.apache.nifi.action.Action; +import org.apache.nifi.action.Component; +import org.apache.nifi.action.Operation; +import org.apache.nifi.action.component.details.ExtensionDetails; +import org.apache.nifi.action.details.ActionDetails; +import org.apache.nifi.action.details.ConfigureDetails; +import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.controller.ReportingTaskNode; +import org.apache.nifi.controller.ScheduledState; +import org.apache.nifi.web.security.user.NiFiUserUtils; +import org.apache.nifi.user.NiFiUser; +import org.apache.nifi.web.api.dto.ReportingTaskDTO; +import org.apache.nifi.web.dao.ReportingTaskDAO; +import org.aspectj.lang.ProceedingJoinPoint; +import org.aspectj.lang.annotation.Around; +import org.aspectj.lang.annotation.Aspect; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Audits reporting creation/removal and configuration changes. + */ +@Aspect +public class ReportingTaskAuditor extends NiFiAuditor { + + private static final Logger logger = LoggerFactory.getLogger(ReportingTaskAuditor.class); + + private static final String NAME = "Name"; + private static final String ANNOTATION_DATA = "Annotation Data"; + + /** + * Audits the creation of reporting task via createReportingTask(). + * + * This method only needs to be run 'after returning'. However, in Java 7 + * the order in which these methods are returned from + * Class.getDeclaredMethods (even though there is no order guaranteed) seems + * to differ from Java 6. SpringAOP depends on this ordering to determine + * advice precedence. By normalizing all advice into Around advice we can + * alleviate this issue. + * + * @param proceedingJoinPoint + * @return + * @throws java.lang.Throwable + */ + @Around("within(org.apache.nifi.web.dao.ReportingTaskDAO+) && " + + "execution(org.apache.nifi.controller.ReportingTaskNode createReportingTask(org.apache.nifi.web.api.dto.ReportingTaskDTO))") + public ReportingTaskNode createReportingTaskAdvice(ProceedingJoinPoint proceedingJoinPoint) throws Throwable { + // update the reporting task state + ReportingTaskNode reportingTask = (ReportingTaskNode) proceedingJoinPoint.proceed(); + + // if no exceptions were thrown, add the reporting task action... + final Action action = generateAuditRecord(reportingTask, Operation.Add); + + // save the actions + if (action != null) { + saveAction(action, logger); + } + + return reportingTask; + } + + /** + * Audits the configuration of a reporting task. + * + * @param proceedingJoinPoint + * @param reportingTaskDTO + * @param reportingTaskDAO + * @return + * @throws Throwable + */ + @Around("within(org.apache.nifi.web.dao.ReportingTaskDAO+) && " + + "execution(org.apache.nifi.controller.ReportingTaskNode updateReportingTask(org.apache.nifi.web.api.dto.ReportingTaskDTO)) && " + + "args(reportingTaskDTO) && " + + "target(reportingTaskDAO)") + public Object updateReportingTaskAdvice(ProceedingJoinPoint proceedingJoinPoint, ReportingTaskDTO reportingTaskDTO, ReportingTaskDAO reportingTaskDAO) throws Throwable { + // determine the initial values for each property/setting thats changing + ReportingTaskNode reportingTask = reportingTaskDAO.getReportingTask(reportingTaskDTO.getId()); + final Map values = extractConfiguredPropertyValues(reportingTask, reportingTaskDTO); + final ScheduledState scheduledState = reportingTask.getScheduledState(); + + // update the reporting task state + final ReportingTaskNode updatedReportingTask = (ReportingTaskNode) proceedingJoinPoint.proceed(); + + // if no exceptions were thrown, add the reporting task action... + reportingTask = reportingTaskDAO.getReportingTask(updatedReportingTask.getIdentifier()); + + // get the current user + final NiFiUser user = NiFiUserUtils.getNiFiUser(); + + // ensure the user was found + if (user != null) { + // determine the updated values + Map updatedValues = extractConfiguredPropertyValues(reportingTask, reportingTaskDTO); + + // create the reporting task details + ExtensionDetails taskDetails = new ExtensionDetails(); + taskDetails.setType(reportingTask.getReportingTask().getClass().getSimpleName()); + + // create a reporting task action + Date actionTimestamp = new Date(); + Collection actions = new ArrayList<>(); + + // go through each updated value + for (String property : updatedValues.keySet()) { + String newValue = updatedValues.get(property); + String oldValue = values.get(property); + Operation operation = null; + + // determine the type of operation + if (oldValue == null || newValue == null || !newValue.equals(oldValue)) { + operation = Operation.Configure; + } + + // create a configuration action accordingly + if (operation != null) { + // clear the value if this property is sensitive + final PropertyDescriptor propertyDescriptor = reportingTask.getReportingTask().getPropertyDescriptor(property); + if (propertyDescriptor != null && propertyDescriptor.isSensitive()) { + if (newValue != null) { + newValue = "********"; + } + if (oldValue != null) { + oldValue = "********"; + } + } else if (ANNOTATION_DATA.equals(property)) { + if (newValue != null) { + newValue = ""; + } + if (oldValue != null) { + oldValue = ""; + } + } + + final ConfigureDetails actionDetails = new ConfigureDetails(); + actionDetails.setName(property); + actionDetails.setValue(newValue); + actionDetails.setPreviousValue(oldValue); + + // create a configuration action + Action configurationAction = new Action(); + configurationAction.setUserDn(user.getDn()); + configurationAction.setUserName(user.getUserName()); + configurationAction.setOperation(operation); + configurationAction.setTimestamp(actionTimestamp); + configurationAction.setSourceId(reportingTask.getIdentifier()); + configurationAction.setSourceName(reportingTask.getName()); + configurationAction.setSourceType(Component.ReportingTask); + configurationAction.setComponentDetails(taskDetails); + configurationAction.setActionDetails(actionDetails); + actions.add(configurationAction); + } + } + + // determine the new executing state + final ScheduledState updatedScheduledState = reportingTask.getScheduledState(); + + // determine if the running state has changed and its not disabled + if (scheduledState != updatedScheduledState) { + // create a reporting task action + Action taskAction = new Action(); + taskAction.setUserDn(user.getDn()); + taskAction.setUserName(user.getUserName()); + taskAction.setTimestamp(new Date()); + taskAction.setSourceId(reportingTask.getIdentifier()); + taskAction.setSourceName(reportingTask.getName()); + taskAction.setSourceType(Component.ReportingTask); + taskAction.setComponentDetails(taskDetails); + + // set the operation accordingly + if (ScheduledState.RUNNING.equals(updatedScheduledState)) { + taskAction.setOperation(Operation.Start); + } else if (ScheduledState.DISABLED.equals(updatedScheduledState)) { + taskAction.setOperation(Operation.Disable); + } else { + // state is now stopped... consider the previous state + if (ScheduledState.RUNNING.equals(scheduledState)) { + taskAction.setOperation(Operation.Stop); + } else if (ScheduledState.DISABLED.equals(scheduledState)) { + taskAction.setOperation(Operation.Enable); + } + } + actions.add(taskAction); + } + + // ensure there are actions to record + if (!actions.isEmpty()) { + // save the actions + saveActions(actions, logger); + } + } + + return updatedReportingTask; + } + + /** + * Audits the removal of a reporting task via deleteReportingTask(). + * + * @param proceedingJoinPoint + * @param reportingTaskId + * @param reportingTaskDAO + * @throws Throwable + */ + @Around("within(org.apache.nifi.web.dao.ReportingTaskDAO+) && " + + "execution(void deleteReportingTask(java.lang.String)) && " + + "args(reportingTaskId) && " + + "target(reportingTaskDAO)") + public void removeReportingTaskAdvice(ProceedingJoinPoint proceedingJoinPoint, String reportingTaskId, ReportingTaskDAO reportingTaskDAO) throws Throwable { + // get the reporting task before removing it + ReportingTaskNode reportingTask = reportingTaskDAO.getReportingTask(reportingTaskId); + + // remove the reporting task + proceedingJoinPoint.proceed(); + + // if no exceptions were thrown, add removal actions... + // audit the reporting task removal + final Action action = generateAuditRecord(reportingTask, Operation.Remove); + + // save the actions + if (action != null) { + saveAction(action, logger); + } + } + + /** + * Generates an audit record for the creation of a reporting task. + * + * @param reportingTask + * @param operation + * @return + */ + public Action generateAuditRecord(ReportingTaskNode reportingTask, Operation operation) { + return generateAuditRecord(reportingTask, operation, null); + } + + /** + * Generates an audit record for the creation of a reporting task. + * + * @param reportingTask + * @param operation + * @param actionDetails + * @return + */ + public Action generateAuditRecord(ReportingTaskNode reportingTask, Operation operation, ActionDetails actionDetails) { + Action action = null; + + // get the current user + NiFiUser user = NiFiUserUtils.getNiFiUser(); + + // ensure the user was found + if (user != null) { + // create the reporting task details + ExtensionDetails taskDetails = new ExtensionDetails(); + taskDetails.setType(reportingTask.getReportingTask().getClass().getSimpleName()); + + // create the reporting task action for adding this reporting task + action = new Action(); + action.setUserDn(user.getDn()); + action.setUserName(user.getUserName()); + action.setOperation(operation); + action.setTimestamp(new Date()); + action.setSourceId(reportingTask.getIdentifier()); + action.setSourceName(reportingTask.getName()); + action.setSourceType(Component.ReportingTask); + action.setComponentDetails(taskDetails); + + if (actionDetails != null) { + action.setActionDetails(actionDetails); + } + } + + return action; + } + + /** + * Extracts the values for the configured properties from the specified ReportingTask. + * + * @param reportingTask + * @param reportingTaskDTO + * @return + */ + private Map extractConfiguredPropertyValues(ReportingTaskNode reportingTask, ReportingTaskDTO reportingTaskDTO) { + Map values = new HashMap<>(); + + if (reportingTaskDTO.getName() != null) { + values.put(NAME, reportingTask.getName()); + } + if (reportingTaskDTO.getAnnotationData() != null) { + values.put(ANNOTATION_DATA, reportingTask.getAnnotationData()); + } + if (reportingTaskDTO.getProperties() != null) { + // for each property specified, extract its configured value + Map properties = reportingTaskDTO.getProperties(); + Map configuredProperties = reportingTask.getProperties(); + for (String propertyName : properties.keySet()) { + // build a descriptor for getting the configured value + PropertyDescriptor propertyDescriptor = new PropertyDescriptor.Builder().name(propertyName).build(); + String configuredPropertyValue = configuredProperties.get(propertyDescriptor); + + // if the configured value couldn't be found, use the default value from the actual descriptor + if (configuredPropertyValue == null) { + propertyDescriptor = locatePropertyDescriptor(configuredProperties.keySet(), propertyDescriptor); + configuredPropertyValue = propertyDescriptor.getDefaultValue(); + } + values.put(propertyName, configuredPropertyValue); + } + } + + return values; + } + + /** + * Locates the actual property descriptor for the given spec property + * descriptor. + * + * @param propertyDescriptors + * @param specDescriptor + * @return + */ + private PropertyDescriptor locatePropertyDescriptor(Set propertyDescriptors, PropertyDescriptor specDescriptor) { + for (PropertyDescriptor propertyDescriptor : propertyDescriptors) { + if (propertyDescriptor.equals(specDescriptor)) { + return propertyDescriptor; + } + } + return specDescriptor; + } + +} diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/SnippetAuditor.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/SnippetAuditor.java index 9c075e518a..1f8942b4f2 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/SnippetAuditor.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/SnippetAuditor.java @@ -25,7 +25,7 @@ import java.util.Set; import org.apache.nifi.action.Action; import org.apache.nifi.action.Component; import org.apache.nifi.action.Operation; -import org.apache.nifi.action.component.details.ProcessorDetails; +import org.apache.nifi.action.component.details.ExtensionDetails; import org.apache.nifi.action.component.details.RemoteProcessGroupDetails; import org.apache.nifi.action.details.ConnectDetails; import org.apache.nifi.connectable.ConnectableType; @@ -94,7 +94,7 @@ public class SnippetAuditor extends NiFiAuditor { */ @Around("within(org.apache.nifi.web.dao.SnippetDAO+) && " + "execution(org.apache.nifi.web.api.dto.FlowSnippetDTO copySnippet(java.lang.String, java.lang.String, java.lang.Double, java.lang.Double))") - public Object copySnippetAdvice(ProceedingJoinPoint proceedingJoinPoint) throws Throwable { + public FlowSnippetDTO copySnippetAdvice(ProceedingJoinPoint proceedingJoinPoint) throws Throwable { // perform the underlying operation FlowSnippetDTO snippet = (FlowSnippetDTO) proceedingJoinPoint.proceed(); auditSnippet(snippet); @@ -110,7 +110,7 @@ public class SnippetAuditor extends NiFiAuditor { */ @Around("within(org.apache.nifi.web.dao.TemplateDAO+) && " + "execution(org.apache.nifi.web.api.dto.FlowSnippetDTO instantiateTemplate(java.lang.String, java.lang.Double, java.lang.Double, java.lang.String))") - public Object instantiateTemplateAdvice(ProceedingJoinPoint proceedingJoinPoint) throws Throwable { + public FlowSnippetDTO instantiateTemplateAdvice(ProceedingJoinPoint proceedingJoinPoint) throws Throwable { // perform the underlying operation FlowSnippetDTO snippet = (FlowSnippetDTO) proceedingJoinPoint.proceed(); auditSnippet(snippet); @@ -153,7 +153,7 @@ public class SnippetAuditor extends NiFiAuditor { // processors for (final ProcessorDTO processor : snippet.getProcessors()) { - final ProcessorDetails processorDetails = new ProcessorDetails(); + final ExtensionDetails processorDetails = new ExtensionDetails(); processorDetails.setType(StringUtils.substringAfterLast(processor.getType(), ".")); final Action action = generateAuditRecord(processor.getId(), processor.getName(), Component.Processor, Operation.Add, timestamp); @@ -256,15 +256,16 @@ public class SnippetAuditor extends NiFiAuditor { * Audits a bulk move. * * @param proceedingJoinPoint - * @param snippetId + * @param snippetDTO * @param snippetDAO + * @return * @throws Throwable */ @Around("within(org.apache.nifi.web.dao.SnippetDAO+) && " + "execution(org.apache.nifi.controller.Snippet updateSnippet(org.apache.nifi.web.api.dto.SnippetDTO)) && " + "args(snippetDTO) && " + "target(snippetDAO)") - public Object updateSnippetAdvice(ProceedingJoinPoint proceedingJoinPoint, SnippetDTO snippetDTO, SnippetDAO snippetDAO) throws Throwable { + public Snippet updateSnippetAdvice(ProceedingJoinPoint proceedingJoinPoint, SnippetDTO snippetDTO, SnippetDAO snippetDAO) throws Throwable { // get the snippet before removing it Snippet snippet = snippetDAO.getSnippet(snippetDTO.getId()); final String previousGroupId = snippet.getParentGroupId(); diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/NiFiServiceFacade.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/NiFiServiceFacade.java index 63d302c1cb..e3afbf92e3 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/NiFiServiceFacade.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/NiFiServiceFacade.java @@ -19,14 +19,17 @@ package org.apache.nifi.web; import java.util.Collection; import java.util.Date; import java.util.Set; +import org.apache.nifi.controller.ScheduledState; import org.apache.nifi.controller.repository.claim.ContentDirection; +import org.apache.nifi.controller.service.ControllerServiceState; import org.apache.nifi.web.api.dto.BulletinBoardDTO; import org.apache.nifi.web.api.dto.BulletinQueryDTO; import org.apache.nifi.web.api.dto.ClusterDTO; import org.apache.nifi.web.api.dto.ConnectionDTO; import org.apache.nifi.web.api.dto.ControllerConfigurationDTO; import org.apache.nifi.web.api.dto.ControllerDTO; +import org.apache.nifi.web.api.dto.ControllerServiceDTO; import org.apache.nifi.web.api.dto.CounterDTO; import org.apache.nifi.web.api.dto.CountersDTO; import org.apache.nifi.web.api.dto.DocumentedTypeDTO; @@ -38,9 +41,12 @@ import org.apache.nifi.web.api.dto.NodeSystemDiagnosticsDTO; import org.apache.nifi.web.api.dto.PortDTO; import org.apache.nifi.web.api.dto.ProcessGroupDTO; import org.apache.nifi.web.api.dto.ProcessorDTO; -import org.apache.nifi.web.api.dto.ProcessorHistoryDTO; +import org.apache.nifi.web.api.dto.ComponentHistoryDTO; +import org.apache.nifi.web.api.dto.ControllerServiceReferencingComponentDTO; +import org.apache.nifi.web.api.dto.PropertyDescriptorDTO; import org.apache.nifi.web.api.dto.RemoteProcessGroupDTO; import org.apache.nifi.web.api.dto.RemoteProcessGroupPortDTO; +import org.apache.nifi.web.api.dto.ReportingTaskDTO; import org.apache.nifi.web.api.dto.RevisionDTO; import org.apache.nifi.web.api.dto.SnippetDTO; import org.apache.nifi.web.api.dto.SystemDiagnosticsDTO; @@ -242,7 +248,21 @@ public interface NiFiServiceFacade { * @return The list of available processor types */ Set getProcessorTypes(); - + + /** + * Returns the list of controller service types. + * + * @return The list of available controller types + */ + Set getControllerServiceTypes(); + + /** + * Returns the list of reporting task types. + * + * @return The list of available reporting task types + */ + Set getReportingTaskTypes(); + /** * Returns the list of prioritizer types. * @@ -359,6 +379,16 @@ public interface NiFiServiceFacade { */ StatusHistoryDTO getProcessorStatusHistory(String groupId, String id); + /** + * Get the descriptor for the specified property of the specified processor. + * + * @param groupId + * @param id + * @param property + * @return + */ + PropertyDescriptorDTO getProcessorPropertyDescriptor(String groupId, String id, String property); + /** * Gets all the Processor transfer objects for this controller. * @@ -923,6 +953,172 @@ public interface NiFiServiceFacade { */ ConfigurationSnapshot deleteLabel(Revision revision, String groupId, String labelId); + // ---------------------------------------- + // Controller Services methods + // ---------------------------------------- + + /** + * Creates a controller service. + * + * @param revision Revision to compare with current base revision + * @param controllerServiceDTO The controller service DTO + * @return The controller service DTO + */ + ConfigurationSnapshot createControllerService(Revision revision, ControllerServiceDTO controllerServiceDTO); + + /** + * Gets all controller services. + * + * @return + */ + Set getControllerServices(); + + /** + * Gets the specified controller service. + * + * @param controllerServiceId + * @return + */ + ControllerServiceDTO getControllerService(String controllerServiceId); + + /** + * Get the descriptor for the specified property of the specified controller service. + * + * @param id + * @param property + * @return + */ + PropertyDescriptorDTO getControllerServicePropertyDescriptor(String id, String property); + + /** + * Gets the references for specified controller service. + * + * @param controllerServiceId + * @return + */ + Set getControllerServiceReferencingComponents(String controllerServiceId); + + /** + * Updates the referencing components for the specified controller service. + * + * @param revision + * @param controllerServiceId + * @param scheduledState + * @param controllerServiceState the value of state + * @return The referencing component dtos + */ + ConfigurationSnapshot> updateControllerServiceReferencingComponents(Revision revision, String controllerServiceId, ScheduledState scheduledState, ControllerServiceState controllerServiceState); + + /** + * Updates the specified label. + * + * @param revision Revision to compare with current base revision + * @param controllerServiceDTO The controller service DTO + * @return The controller service DTO + */ + ConfigurationSnapshot updateControllerService(Revision revision, ControllerServiceDTO controllerServiceDTO); + + /** + * Deletes the specified label. + * + * @param revision Revision to compare with current base revision + * @param controllerServiceId The controller service id + * @return + */ + ConfigurationSnapshot deleteControllerService(Revision revision, String controllerServiceId); + + /** + * Verifies the specified controller service can be updated. + * + * @param controllerServiceDTO + */ + void verifyUpdateControllerService(ControllerServiceDTO controllerServiceDTO); + + /** + * Verifies the referencing components of the specified controller service can be updated. + * + * @param controllerServiceId + * @param scheduledState + * @param controllerServiceState + */ + void verifyUpdateControllerServiceReferencingComponents(String controllerServiceId, ScheduledState scheduledState, ControllerServiceState controllerServiceState); + + /** + * Verifies the specified controller service can be removed. + * + * @param controllerServiceId + */ + void verifyDeleteControllerService(String controllerServiceId); + + // ---------------------------------------- + // Reporting Task methods + // ---------------------------------------- + + /** + * Creates a reporting task. + * + * @param revision Revision to compare with current base revision + * @param reportingTaskDTO The reporting task DTO + * @return The reporting task DTO + */ + ConfigurationSnapshot createReportingTask(Revision revision, ReportingTaskDTO reportingTaskDTO); + + /** + * Gets all reporting tasks. + * + * @return + */ + Set getReportingTasks(); + + /** + * Gets the specified reporting task. + * + * @param reportingTaskId + * @return + */ + ReportingTaskDTO getReportingTask(String reportingTaskId); + + /** + * Get the descriptor for the specified property of the specified reporting task. + * + * @param id + * @param property + * @return + */ + PropertyDescriptorDTO getReportingTaskPropertyDescriptor(String id, String property); + + /** + * Updates the specified reporting task. + * + * @param revision Revision to compare with current base revision + * @param reportingTaskDTO The reporting task DTO + * @return The reporting task DTO + */ + ConfigurationSnapshot updateReportingTask(Revision revision, ReportingTaskDTO reportingTaskDTO); + + /** + * Deletes the specified reporting task. + * + * @param revision Revision to compare with current base revision + * @param reportingTaskId The reporting task id + * @return + */ + ConfigurationSnapshot deleteReportingTask(Revision revision, String reportingTaskId); + + /** + * Verifies the specified reporting task can be updated. + * + * @param reportingTaskDTO + */ + void verifyUpdateReportingTask(ReportingTaskDTO reportingTaskDTO); + + /** + * Verifies the specified reporting task can be removed. + * + * @param reportingTaskId + */ + void verifyDeleteReportingTask(String reportingTaskId); + // ---------------------------------------- // History methods // ---------------------------------------- @@ -950,12 +1146,12 @@ public interface NiFiServiceFacade { void deleteActions(Date endDate); /** - * Gets the history for the specified property for the specified processor. + * Gets the history for the specified property for the specified component. * - * @param processorId + * @param componentId * @return */ - ProcessorHistoryDTO getProcessorHistory(String processorId); + ComponentHistoryDTO getComponentHistory(String componentId); // ---------------------------------------- // Snippet methods diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiServiceFacade.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiServiceFacade.java index 14640d8443..086c46b62f 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiServiceFacade.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiServiceFacade.java @@ -106,7 +106,7 @@ import org.apache.nifi.web.api.dto.PreviousValueDTO; import org.apache.nifi.web.api.dto.ProcessGroupDTO; import org.apache.nifi.web.api.dto.ProcessorConfigDTO; import org.apache.nifi.web.api.dto.ProcessorDTO; -import org.apache.nifi.web.api.dto.ProcessorHistoryDTO; +import org.apache.nifi.web.api.dto.ComponentHistoryDTO; import org.apache.nifi.web.api.dto.PropertyHistoryDTO; import org.apache.nifi.web.api.dto.RemoteProcessGroupDTO; import org.apache.nifi.web.api.dto.RemoteProcessGroupPortDTO; @@ -152,6 +152,19 @@ import org.apache.nifi.web.util.SnippetUtils; import org.apache.commons.collections4.CollectionUtils; import org.apache.commons.lang3.StringUtils; +import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.components.Validator; +import org.apache.nifi.controller.ReportingTaskNode; +import org.apache.nifi.controller.ScheduledState; +import org.apache.nifi.controller.service.ControllerServiceNode; +import org.apache.nifi.controller.service.ControllerServiceReference; +import org.apache.nifi.controller.service.ControllerServiceState; +import org.apache.nifi.web.api.dto.ControllerServiceDTO; +import org.apache.nifi.web.api.dto.ControllerServiceReferencingComponentDTO; +import org.apache.nifi.web.api.dto.PropertyDescriptorDTO; +import org.apache.nifi.web.api.dto.ReportingTaskDTO; +import org.apache.nifi.web.dao.ControllerServiceDAO; +import org.apache.nifi.web.dao.ReportingTaskDAO; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.springframework.security.access.AccessDeniedException; @@ -162,8 +175,6 @@ import org.springframework.security.access.AccessDeniedException; public class StandardNiFiServiceFacade implements NiFiServiceFacade { private static final Logger logger = LoggerFactory.getLogger(StandardNiFiServiceFacade.class); - private static final String INVALID_REVISION_ERROR = "Given revision %s does not match current revision %s."; - private static final String SYNC_ERROR = "This NiFi instance has been updated by '%s'. Please refresh to synchronize the view."; // nifi core components private ControllerFacade controllerFacade; @@ -182,6 +193,8 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade { private PortDAO inputPortDAO; private PortDAO outputPortDAO; private ConnectionDAO connectionDAO; + private ControllerServiceDAO controllerServiceDAO; + private ReportingTaskDAO reportingTaskDAO; private TemplateDAO templateDAO; // administrative services @@ -195,54 +208,10 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade { private NiFiProperties properties; private DtoFactory dtoFactory; - /** - * Checks the specified revision against the current revision. - * - * @param revision The revision to check - * @param clientId The client id - * @return Whether or not the request should proceed - * @throws NiFiCoreException If the specified revision is not current - */ - private void checkRevision(Revision revision) { - - boolean approved = optimisticLockingManager.isCurrent(revision); - - if (!approved) { - Revision currentRevision = optimisticLockingManager.getRevision(); - logger.debug("Revision check failed because current revision is " + currentRevision + " but supplied revision is " + revision); - - if (StringUtils.isBlank(currentRevision.getClientId()) || currentRevision.getVersion() == null) { - throw new InvalidRevisionException(String.format(INVALID_REVISION_ERROR, revision, currentRevision)); - } else { - throw new InvalidRevisionException(String.format(SYNC_ERROR, optimisticLockingManager.getLastModifier())); - } - } - } - - /** - * Increments the revision and updates the last modifier. - * - * @param revision - * @return - */ - private Revision updateRevision(Revision revision) { - // update the client id and modifier - final Revision updatedRevision = optimisticLockingManager.incrementRevision(revision.getClientId()); - - // get the nifi user to extract the username - NiFiUser user = NiFiUserUtils.getNiFiUser(); - if (user == null) { - optimisticLockingManager.setLastModifier("unknown"); - } else { - optimisticLockingManager.setLastModifier(user.getUserName()); - } - - return updatedRevision; - } - // ----------------------------------------- // Verification Operations // ----------------------------------------- + @Override public void verifyCreateConnection(String groupId, ConnectionDTO connectionDTO) { connectionDAO.verifyCreate(groupId, connectionDTO); @@ -360,100 +329,123 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade { remoteProcessGroupDAO.verifyDelete(groupId, remoteProcessGroupId); } + @Override + public void verifyUpdateControllerService(ControllerServiceDTO controllerServiceDTO) { + // if service does not exist, then the update request is likely creating it + // so we don't verify since it will fail + if (controllerServiceDAO.hasControllerService(controllerServiceDTO.getId())) { + controllerServiceDAO.verifyUpdate(controllerServiceDTO); + } + } + + @Override + public void verifyUpdateControllerServiceReferencingComponents(String controllerServiceId, ScheduledState scheduledState, ControllerServiceState controllerServiceState) { + controllerServiceDAO.verifyUpdateReferencingComponents(controllerServiceId, scheduledState, controllerServiceState); + } + + @Override + public void verifyDeleteControllerService(String controllerServiceId) { + controllerServiceDAO.verifyDelete(controllerServiceId); + } + + @Override + public void verifyUpdateReportingTask(ReportingTaskDTO reportingTaskDTO) { + // if tasks does not exist, then the update request is likely creating it + // so we don't verify since it will fail + if (reportingTaskDAO.hasReportingTask(reportingTaskDTO.getId())) { + reportingTaskDAO.verifyUpdate(reportingTaskDTO); + } + } + + @Override + public void verifyDeleteReportingTask(String reportingTaskId) { + reportingTaskDAO.verifyDelete(reportingTaskId); + } + // ----------------------------------------- // Write Operations // ----------------------------------------- + @Override - public ConfigurationSnapshot updateConnection(Revision revision, String groupId, ConnectionDTO connectionDTO) { - - // ensure the proper revision before performing the update - checkRevision(revision); - + public ConfigurationSnapshot updateConnection(final Revision revision, final String groupId, final ConnectionDTO connectionDTO) { // if connection does not exist, then create new connection if (connectionDAO.hasConnection(groupId, connectionDTO.getId()) == false) { return createConnection(revision, groupId, connectionDTO); } + + return optimisticLockingManager.configureFlow(revision, new ConfigurationRequest() { + @Override + public ConnectionDTO execute() { + final Connection connection = connectionDAO.updateConnection(groupId, connectionDTO); - final Connection connection = connectionDAO.updateConnection(groupId, connectionDTO); - - // update the revision and generate a response - final Revision updatedRevision = updateRevision(revision); - final ConfigurationSnapshot response = new ConfigurationSnapshot<>(updatedRevision.getVersion(), dtoFactory.createConnectionDto(connection)); - - // save the flow - controllerFacade.save(); - - return response; + controllerFacade.save(); + + return dtoFactory.createConnectionDto(connection); + } + }); } @Override - public ConfigurationSnapshot updateProcessor(Revision revision, String groupId, ProcessorDTO processorDTO) { - - // ensure the proper revision before performing the update - checkRevision(revision); - + public ConfigurationSnapshot updateProcessor(final Revision revision, final String groupId, final ProcessorDTO processorDTO) { // if processor does not exist, then create new processor if (processorDAO.hasProcessor(groupId, processorDTO.getId()) == false) { return createProcessor(revision, groupId, processorDTO); } + + return optimisticLockingManager.configureFlow(revision, new ConfigurationRequest() { + @Override + public ProcessorDTO execute() { + // update the processor + ProcessorNode processor = processorDAO.updateProcessor(groupId, processorDTO); - // update the processor - ProcessorNode processor = processorDAO.updateProcessor(groupId, processorDTO); - - // update the revision and generate a response - final Revision updatedRevision = updateRevision(revision); - final ConfigurationSnapshot response = new ConfigurationSnapshot<>(updatedRevision.getVersion(), dtoFactory.createProcessorDto(processor)); - - // save the flow - controllerFacade.save(); - - return response; + // save the flow + controllerFacade.save(); + + return dtoFactory.createProcessorDto(processor); + } + }); } @Override - public ConfigurationSnapshot updateLabel(Revision revision, String groupId, LabelDTO labelDTO) { - // ensure the proper revision before performing the update - checkRevision(revision); - + public ConfigurationSnapshot updateLabel(final Revision revision, final String groupId, final LabelDTO labelDTO) { // if label does not exist, then create new label if (labelDAO.hasLabel(groupId, labelDTO.getId()) == false) { return createLabel(revision, groupId, labelDTO); } + + return optimisticLockingManager.configureFlow(revision, new ConfigurationRequest() { + @Override + public LabelDTO execute() { + // update the existing label + final Label label = labelDAO.updateLabel(groupId, labelDTO); - // update the existing label - final Label label = labelDAO.updateLabel(groupId, labelDTO); - - // update the revision and generate a response - final Revision updatedRevision = updateRevision(revision); - final ConfigurationSnapshot response = new ConfigurationSnapshot<>(updatedRevision.getVersion(), dtoFactory.createLabelDto(label)); - - // save updated controller - controllerFacade.save(); - - return response; + // save updated controller + controllerFacade.save(); + + return dtoFactory.createLabelDto(label); + } + }); } @Override - public ConfigurationSnapshot updateFunnel(Revision revision, String groupId, FunnelDTO funnelDTO) { - // ensure the proper revision before performing the update - checkRevision(revision); - + public ConfigurationSnapshot updateFunnel(final Revision revision, final String groupId, final FunnelDTO funnelDTO) { // if label does not exist, then create new label if (funnelDAO.hasFunnel(groupId, funnelDTO.getId()) == false) { return createFunnel(revision, groupId, funnelDTO); } + + return optimisticLockingManager.configureFlow(revision, new ConfigurationRequest() { + @Override + public FunnelDTO execute() { + // update the existing label + final Funnel funnel = funnelDAO.updateFunnel(groupId, funnelDTO); - // update the existing label - final Funnel funnel = funnelDAO.updateFunnel(groupId, funnelDTO); - - // update the revision and generate a response - final Revision updatedRevision = updateRevision(revision); - final ConfigurationSnapshot response = new ConfigurationSnapshot<>(updatedRevision.getVersion(), dtoFactory.createFunnelDto(funnel)); - - // save updated controller - controllerFacade.save(); - - return response; + // save updated controller + controllerFacade.save(); + + return dtoFactory.createFunnelDto(funnel); + } + }); } @Override @@ -466,141 +458,126 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade { } @Override - public ConfigurationSnapshot updateSnippet(Revision revision, SnippetDTO snippetDto) { - // ensure the proper revision before performing the update - checkRevision(revision); - + public ConfigurationSnapshot updateSnippet(final Revision revision, final SnippetDTO snippetDto) { // if label does not exist, then create new label if (snippetDAO.hasSnippet(snippetDto.getId()) == false) { return createSnippet(revision, snippetDto); } + + return optimisticLockingManager.configureFlow(revision, new ConfigurationRequest() { + @Override + public SnippetDTO execute() { + // update the snippet + final Snippet snippet = snippetDAO.updateSnippet(snippetDto); - // update the snippet - final Snippet snippet = snippetDAO.updateSnippet(snippetDto); + // build the snippet dto + final SnippetDTO responseSnippetDto = dtoFactory.createSnippetDto(snippet); + responseSnippetDto.setContents(snippetUtils.populateFlowSnippet(snippet, false)); - // build the snippet dto - final SnippetDTO responseSnippetDto = dtoFactory.createSnippetDto(snippet); - responseSnippetDto.setContents(snippetUtils.populateFlowSnippet(snippet, false)); - - // update the revision and generate a response - final Revision updatedRevision = updateRevision(revision); - final ConfigurationSnapshot response = new ConfigurationSnapshot<>(updatedRevision.getVersion(), responseSnippetDto); - - // save updated controller if applicable - if (snippetDto.getParentGroupId() != null && snippet.isLinked()) { - controllerFacade.save(); - } - - return response; + // save updated controller if applicable + if (snippetDto.getParentGroupId() != null && snippet.isLinked()) { + controllerFacade.save(); + } + + return responseSnippetDto; + } + }); } @Override - public ConfigurationSnapshot updateInputPort(Revision revision, String groupId, PortDTO inputPortDTO) { - // ensure the proper revision before performing the update - checkRevision(revision); - + public ConfigurationSnapshot updateInputPort(final Revision revision, final String groupId, final PortDTO inputPortDTO) { // if input port does not exist, then create new input port if (inputPortDAO.hasPort(groupId, inputPortDTO.getId()) == false) { return createInputPort(revision, groupId, inputPortDTO); } + + return optimisticLockingManager.configureFlow(revision, new ConfigurationRequest() { + @Override + public PortDTO execute() { + final Port inputPort = inputPortDAO.updatePort(groupId, inputPortDTO); - final Port inputPort = inputPortDAO.updatePort(groupId, inputPortDTO); - - // update the revision and generate a response - final Revision updatedRevision = updateRevision(revision); - final ConfigurationSnapshot response = new ConfigurationSnapshot<>(updatedRevision.getVersion(), dtoFactory.createPortDto(inputPort)); - - // save updated controller - controllerFacade.save(); - - return response; + // save updated controller + controllerFacade.save(); + + return dtoFactory.createPortDto(inputPort); + } + }); } @Override - public ConfigurationSnapshot updateOutputPort(Revision revision, String groupId, PortDTO outputPortDTO) { - // ensure the proper revision before performing the update - checkRevision(revision); - + public ConfigurationSnapshot updateOutputPort(final Revision revision, final String groupId, final PortDTO outputPortDTO) { // if output port does not exist, then create new output port if (outputPortDAO.hasPort(groupId, outputPortDTO.getId()) == false) { return createOutputPort(revision, groupId, outputPortDTO); } + + return optimisticLockingManager.configureFlow(revision, new ConfigurationRequest() { + @Override + public PortDTO execute() { + final Port outputPort = outputPortDAO.updatePort(groupId, outputPortDTO); - final Port outputPort = outputPortDAO.updatePort(groupId, outputPortDTO); - - // update the revision and generate a response - final Revision updatedRevision = updateRevision(revision); - final ConfigurationSnapshot response = new ConfigurationSnapshot<>(updatedRevision.getVersion(), dtoFactory.createPortDto(outputPort)); - - // save updated controller - controllerFacade.save(); - - return response; + // save updated controller + controllerFacade.save(); + + return dtoFactory.createPortDto(outputPort); + } + }); } @Override - public ConfigurationSnapshot updateRemoteProcessGroup(Revision revision, String groupId, RemoteProcessGroupDTO remoteProcessGroupDTO) { - // ensure the proper revision before performing the update - checkRevision(revision); - + public ConfigurationSnapshot updateRemoteProcessGroup(final Revision revision, final String groupId, final RemoteProcessGroupDTO remoteProcessGroupDTO) { // if controller reference does not exist, then create new controller reference if (remoteProcessGroupDAO.hasRemoteProcessGroup(groupId, remoteProcessGroupDTO.getId()) == false) { return createRemoteProcessGroup(revision, groupId, remoteProcessGroupDTO); } + + return optimisticLockingManager.configureFlow(revision, new ConfigurationRequest() { + @Override + public RemoteProcessGroupDTO execute() { + RemoteProcessGroup remoteProcessGroup = remoteProcessGroupDAO.updateRemoteProcessGroup(groupId, remoteProcessGroupDTO); - RemoteProcessGroup remoteProcessGroup = remoteProcessGroupDAO.updateRemoteProcessGroup(groupId, remoteProcessGroupDTO); - - // update the revision and generate a response - final Revision updatedRevision = updateRevision(revision); - final ConfigurationSnapshot response = new ConfigurationSnapshot<>(updatedRevision.getVersion(), dtoFactory.createRemoteProcessGroupDto(remoteProcessGroup)); - - // save updated controller - controllerFacade.save(); - - return response; + // save updated controller + controllerFacade.save(); + + return dtoFactory.createRemoteProcessGroupDto(remoteProcessGroup); + } + }); } @Override - public ConfigurationSnapshot updateRemoteProcessGroupInputPort(Revision revision, String groupId, String remoteProcessGroupId, RemoteProcessGroupPortDTO remoteProcessGroupPortDTO) { - // ensure the proper revision before performing the update - checkRevision(revision); + public ConfigurationSnapshot updateRemoteProcessGroupInputPort(final Revision revision, final String groupId, final String remoteProcessGroupId, final RemoteProcessGroupPortDTO remoteProcessGroupPortDTO) { + return optimisticLockingManager.configureFlow(revision, new ConfigurationRequest() { + @Override + public RemoteProcessGroupPortDTO execute() { + // update the remote port + RemoteGroupPort remoteGroupPort = remoteProcessGroupDAO.updateRemoteProcessGroupInputPort(groupId, remoteProcessGroupId, remoteProcessGroupPortDTO); - // update the remote port - RemoteGroupPort remoteGroupPort = remoteProcessGroupDAO.updateRemoteProcessGroupInputPort(groupId, remoteProcessGroupId, remoteProcessGroupPortDTO); - - // update the revision and generate a response - final Revision updatedRevision = updateRevision(revision); - final ConfigurationSnapshot response = new ConfigurationSnapshot<>(updatedRevision.getVersion(), dtoFactory.createRemoteProcessGroupPortDto(remoteGroupPort)); - - // save updated controller - controllerFacade.save(); - - return response; + // save updated controller + controllerFacade.save(); + + return dtoFactory.createRemoteProcessGroupPortDto(remoteGroupPort); + } + }); } @Override - public ConfigurationSnapshot updateRemoteProcessGroupOutputPort(Revision revision, String groupId, String remoteProcessGroupId, RemoteProcessGroupPortDTO remoteProcessGroupPortDTO) { - // ensure the proper revision before performing the update - checkRevision(revision); + public ConfigurationSnapshot updateRemoteProcessGroupOutputPort(final Revision revision, final String groupId, final String remoteProcessGroupId, final RemoteProcessGroupPortDTO remoteProcessGroupPortDTO) { + return optimisticLockingManager.configureFlow(revision, new ConfigurationRequest() { + @Override + public RemoteProcessGroupPortDTO execute() { + // update the remote port + RemoteGroupPort remoteGroupPort = remoteProcessGroupDAO.updateRemoteProcessGroupOutputPort(groupId, remoteProcessGroupId, remoteProcessGroupPortDTO); - // update the remote port - RemoteGroupPort remoteGroupPort = remoteProcessGroupDAO.updateRemoteProcessGroupOutputPort(groupId, remoteProcessGroupId, remoteProcessGroupPortDTO); - - // update the revision and generate a response - final Revision updatedRevision = updateRevision(revision); - final ConfigurationSnapshot response = new ConfigurationSnapshot<>(updatedRevision.getVersion(), dtoFactory.createRemoteProcessGroupPortDto(remoteGroupPort)); - - // save updated controller - controllerFacade.save(); - - return response; + // save updated controller + controllerFacade.save(); + + return dtoFactory.createRemoteProcessGroupPortDto(remoteGroupPort); + } + }); } @Override - public ConfigurationSnapshot updateProcessGroup(Revision revision, String parentGroupId, ProcessGroupDTO processGroupDTO) { - // ensure the proper revision before performing the update - checkRevision(revision); - + public ConfigurationSnapshot updateProcessGroup(final Revision revision, final String parentGroupId, final ProcessGroupDTO processGroupDTO) { // if process group does not exist, then create new process group if (processGroupDAO.hasProcessGroup(processGroupDTO.getId()) == false) { if (parentGroupId == null) { @@ -609,50 +586,49 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade { return createProcessGroup(parentGroupId, revision, processGroupDTO); } } + + return optimisticLockingManager.configureFlow(revision, new ConfigurationRequest() { + @Override + public ProcessGroupDTO execute() { + // update the process group + ProcessGroup processGroup = processGroupDAO.updateProcessGroup(processGroupDTO); - // update the process group - ProcessGroup processGroup = processGroupDAO.updateProcessGroup(processGroupDTO); - - // update the revision and generate a response - final Revision updatedRevision = updateRevision(revision); - final ConfigurationSnapshot response = new ConfigurationSnapshot<>(updatedRevision.getVersion(), dtoFactory.createProcessGroupDto(processGroup)); - - // save updated controller - controllerFacade.save(); - - return response; + // save updated controller + controllerFacade.save(); + + return dtoFactory.createProcessGroupDto(processGroup); + } + }); } @Override - public ConfigurationSnapshot updateControllerConfiguration(Revision revision, ControllerConfigurationDTO controllerConfigurationDTO) { - // ensure the proper revision before performing the update - checkRevision(revision); + public ConfigurationSnapshot updateControllerConfiguration(final Revision revision, final ControllerConfigurationDTO controllerConfigurationDTO) { + return optimisticLockingManager.configureFlow(revision, new ConfigurationRequest() { + @Override + public ControllerConfigurationDTO execute() { + // update the controller configuration through the proxy + if (controllerConfigurationDTO.getName() != null) { + controllerFacade.setName(controllerConfigurationDTO.getName()); + } + if (controllerConfigurationDTO.getComments() != null) { + controllerFacade.setComments(controllerConfigurationDTO.getComments()); + } + if (controllerConfigurationDTO.getMaxTimerDrivenThreadCount() != null) { + controllerFacade.setMaxTimerDrivenThreadCount(controllerConfigurationDTO.getMaxTimerDrivenThreadCount()); + } + if (controllerConfigurationDTO.getMaxEventDrivenThreadCount() != null) { + controllerFacade.setMaxEventDrivenThreadCount(controllerConfigurationDTO.getMaxEventDrivenThreadCount()); + } - // update the controller configuration through the proxy - if (controllerConfigurationDTO.getName() != null) { - controllerFacade.setName(controllerConfigurationDTO.getName()); - } - if (controllerConfigurationDTO.getComments() != null) { - controllerFacade.setComments(controllerConfigurationDTO.getComments()); - } - if (controllerConfigurationDTO.getMaxTimerDrivenThreadCount() != null) { - controllerFacade.setMaxTimerDrivenThreadCount(controllerConfigurationDTO.getMaxTimerDrivenThreadCount()); - } - if (controllerConfigurationDTO.getMaxEventDrivenThreadCount() != null) { - controllerFacade.setMaxEventDrivenThreadCount(controllerConfigurationDTO.getMaxEventDrivenThreadCount()); - } + // create the controller configuration dto + ControllerConfigurationDTO controllerConfig = getControllerConfiguration(); - // create the controller configuration dto - ControllerConfigurationDTO controllerConfig = getControllerConfiguration(); - - // update the revision and generate a response - final Revision updatedRevision = updateRevision(revision); - final ConfigurationSnapshot response = new ConfigurationSnapshot<>(updatedRevision.getVersion(), controllerConfig); - - // save the flow - controllerFacade.save(); - - return response; + // save the flow + controllerFacade.save(); + + return controllerConfig; + } + }); } @Override @@ -685,74 +661,66 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade { } @Override - public ConfigurationSnapshot deleteConnection(Revision revision, String groupId, String connectionId) { - // ensure the proper revision before performing the update - checkRevision(revision); + public ConfigurationSnapshot deleteConnection(final Revision revision, final String groupId, final String connectionId) { + return optimisticLockingManager.configureFlow(revision, new ConfigurationRequest(){ + @Override + public Void execute() { + connectionDAO.deleteConnection(groupId, connectionId); - connectionDAO.deleteConnection(groupId, connectionId); - - // update the revision and generate a response - final Revision updatedRevision = updateRevision(revision); - final ConfigurationSnapshot response = new ConfigurationSnapshot<>(updatedRevision.getVersion()); - - // save the flow - controllerFacade.save(); - - return response; + // save the flow + controllerFacade.save(); + + return null; + } + }); } @Override - public ConfigurationSnapshot deleteProcessor(Revision revision, String groupId, String processorId) { - // ensure the proper revision before performing the update - checkRevision(revision); + public ConfigurationSnapshot deleteProcessor(final Revision revision, final String groupId, final String processorId) { + return optimisticLockingManager.configureFlow(revision, new ConfigurationRequest() { + @Override + public Void execute() { + // delete the processor and synchronize the connection state + processorDAO.deleteProcessor(groupId, processorId); - // delete the processor and synchronize the connection state - processorDAO.deleteProcessor(groupId, processorId); - - // update the revision and generate a response - final Revision updatedRevision = updateRevision(revision); - final ConfigurationSnapshot response = new ConfigurationSnapshot<>(updatedRevision.getVersion()); - - // save the flow - controllerFacade.save(); - - return response; + // save the flow + controllerFacade.save(); + + return null; + } + }); } @Override - public ConfigurationSnapshot deleteLabel(Revision revision, String groupId, String labelId) { - // ensure the proper revision before performing the update - checkRevision(revision); + public ConfigurationSnapshot deleteLabel(final Revision revision, final String groupId, final String labelId) { + return optimisticLockingManager.configureFlow(revision, new ConfigurationRequest() { + @Override + public Void execute() { + // delete the label + labelDAO.deleteLabel(groupId, labelId); - // delete the label - labelDAO.deleteLabel(groupId, labelId); - - // update the revision and generate a response - final Revision updatedRevision = updateRevision(revision); - final ConfigurationSnapshot response = new ConfigurationSnapshot<>(updatedRevision.getVersion()); - - // save the flow - controllerFacade.save(); - - return response; + // save the flow + controllerFacade.save(); + + return null; + } + }); } @Override - public ConfigurationSnapshot deleteFunnel(Revision revision, String groupId, String funnelId) { - // ensure the proper revision before performing the update - checkRevision(revision); + public ConfigurationSnapshot deleteFunnel(final Revision revision, final String groupId, final String funnelId) { + return optimisticLockingManager.configureFlow(revision, new ConfigurationRequest() { + @Override + public Void execute() { + // delete the label + funnelDAO.deleteFunnel(groupId, funnelId); - // delete the label - funnelDAO.deleteFunnel(groupId, funnelId); - - // update the revision and generate a response - final Revision updatedRevision = updateRevision(revision); - final ConfigurationSnapshot response = new ConfigurationSnapshot<>(updatedRevision.getVersion()); - - // save the flow - controllerFacade.save(); - - return response; + // save the flow + controllerFacade.save(); + + return null; + } + }); } @Override @@ -761,95 +729,85 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade { } @Override - public ConfigurationSnapshot deleteSnippet(Revision revision, String snippetId) { - // ensure the proper revision before performing the update - checkRevision(revision); + public ConfigurationSnapshot deleteSnippet(final Revision revision, final String snippetId) { + return optimisticLockingManager.configureFlow(revision, new ConfigurationRequest() { + @Override + public Void execute() { + // determine if this snippet was linked to the data flow + Snippet snippet = snippetDAO.getSnippet(snippetId); + boolean linked = snippet.isLinked(); - // determine if this snippet was linked to the data flow - Snippet snippet = snippetDAO.getSnippet(snippetId); - boolean linked = snippet.isLinked(); + // delete the snippet + snippetDAO.deleteSnippet(snippetId); - // delete the snippet - snippetDAO.deleteSnippet(snippetId); - - // update the revision and generate a response - final Revision updatedRevision = updateRevision(revision); - final ConfigurationSnapshot response = new ConfigurationSnapshot<>(updatedRevision.getVersion()); - - // save the flow if necessary - if (linked) { - controllerFacade.save(); - } - - return response; + // save the flow if necessary + if (linked) { + controllerFacade.save(); + } + + return null; + } + }); } @Override - public ConfigurationSnapshot deleteInputPort(Revision revision, String groupId, String inputPortId) { - // ensure the proper revision before performing the update - checkRevision(revision); + public ConfigurationSnapshot deleteInputPort(final Revision revision, final String groupId, final String inputPortId) { + return optimisticLockingManager.configureFlow(revision, new ConfigurationRequest() { + @Override + public Void execute() { + inputPortDAO.deletePort(groupId, inputPortId); - inputPortDAO.deletePort(groupId, inputPortId); - - // update the revision and generate a response - final Revision updatedRevision = updateRevision(revision); - final ConfigurationSnapshot response = new ConfigurationSnapshot<>(updatedRevision.getVersion()); - - // save the flow - controllerFacade.save(); - - return response; + // save the flow + controllerFacade.save(); + + return null; + } + }); } @Override - public ConfigurationSnapshot deleteOutputPort(Revision revision, String groupId, String outputPortId) { - // ensure the proper revision before performing the update - checkRevision(revision); + public ConfigurationSnapshot deleteOutputPort(final Revision revision, final String groupId, final String outputPortId) { + return optimisticLockingManager.configureFlow(revision, new ConfigurationRequest() { + @Override + public Void execute() { + outputPortDAO.deletePort(groupId, outputPortId); - outputPortDAO.deletePort(groupId, outputPortId); - - // update the revision and generate a response - final Revision updatedRevision = updateRevision(revision); - final ConfigurationSnapshot response = new ConfigurationSnapshot<>(updatedRevision.getVersion()); - - // save the flow - controllerFacade.save(); - - return response; + // save the flow + controllerFacade.save(); + + return null; + } + }); } @Override - public ConfigurationSnapshot deleteProcessGroup(Revision revision, String groupId) { - // ensure the proper revision before performing the update - checkRevision(revision); + public ConfigurationSnapshot deleteProcessGroup(final Revision revision, final String groupId) { + return optimisticLockingManager.configureFlow(revision, new ConfigurationRequest() { + @Override + public Void execute() { + processGroupDAO.deleteProcessGroup(groupId); - processGroupDAO.deleteProcessGroup(groupId); - - // update the revision and generate a response - final Revision updatedRevision = updateRevision(revision); - final ConfigurationSnapshot response = new ConfigurationSnapshot<>(updatedRevision.getVersion()); - - // save the flow - controllerFacade.save(); - - return response; + // save the flow + controllerFacade.save(); + + return null; + } + }); } @Override - public ConfigurationSnapshot deleteRemoteProcessGroup(Revision revision, String groupId, String remoteProcessGroupId) { - // ensure the proper revision before performing the update - checkRevision(revision); + public ConfigurationSnapshot deleteRemoteProcessGroup(final Revision revision, final String groupId, final String remoteProcessGroupId) { + return optimisticLockingManager.configureFlow(revision, new ConfigurationRequest() { + @Override + public Void execute() { + remoteProcessGroupDAO.deleteRemoteProcessGroup(groupId, remoteProcessGroupId); - remoteProcessGroupDAO.deleteRemoteProcessGroup(groupId, remoteProcessGroupId); - - // update the revision and generate a response - final Revision updatedRevision = updateRevision(revision); - final ConfigurationSnapshot response = new ConfigurationSnapshot<>(updatedRevision.getVersion()); - - // save the flow - controllerFacade.save(); - - return response; + // save the flow + controllerFacade.save(); + + return null; + } + }); } @Override @@ -859,97 +817,86 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade { } @Override - public ConfigurationSnapshot createConnection(Revision revision, String groupId, ConnectionDTO connectionDTO) { + public ConfigurationSnapshot createConnection(final Revision revision, final String groupId, final ConnectionDTO connectionDTO) { + return optimisticLockingManager.configureFlow(revision, new ConfigurationRequest() { + @Override + public ConnectionDTO execute() { + // ensure id is set + if (StringUtils.isBlank(connectionDTO.getId())) { + connectionDTO.setId(UUID.randomUUID().toString()); + } - // ensure the proper revision before performing the update - checkRevision(revision); + final Connection connection = connectionDAO.createConnection(groupId, connectionDTO); - // ensure id is set - if (StringUtils.isBlank(connectionDTO.getId())) { - connectionDTO.setId(UUID.randomUUID().toString()); - } - - final Connection connection = connectionDAO.createConnection(groupId, connectionDTO); - - // update the revision and generate a response - final Revision updatedRevision = updateRevision(revision); - final ConfigurationSnapshot response = new ConfigurationSnapshot<>(updatedRevision.getVersion(), dtoFactory.createConnectionDto(connection)); - - // save the flow - controllerFacade.save(); - - return response; + // save the flow + controllerFacade.save(); + + return dtoFactory.createConnectionDto(connection); + } + }); } @Override - public ConfigurationSnapshot createProcessor(Revision revision, String groupId, ProcessorDTO processorDTO) { + public ConfigurationSnapshot createProcessor(final Revision revision, final String groupId, final ProcessorDTO processorDTO) { + return optimisticLockingManager.configureFlow(revision, new ConfigurationRequest() { + @Override + public ProcessorDTO execute() { + // ensure id is set + if (StringUtils.isBlank(processorDTO.getId())) { + processorDTO.setId(UUID.randomUUID().toString()); + } - // ensure the proper revision before performing the update - checkRevision(revision); + // create the processor + final ProcessorNode processor = processorDAO.createProcessor(groupId, processorDTO); - // ensure id is set - if (StringUtils.isBlank(processorDTO.getId())) { - processorDTO.setId(UUID.randomUUID().toString()); - } - - // create the processor - final ProcessorNode processor = processorDAO.createProcessor(groupId, processorDTO); - - // update the revision and generate a response - final Revision updatedRevision = updateRevision(revision); - final ConfigurationSnapshot response = new ConfigurationSnapshot<>(updatedRevision.getVersion(), dtoFactory.createProcessorDto(processor)); - - // save the flow - controllerFacade.save(); - - return response; + // save the flow + controllerFacade.save(); + + return dtoFactory.createProcessorDto(processor); + } + }); } @Override - public ConfigurationSnapshot createLabel(Revision revision, String groupId, LabelDTO labelDTO) { + public ConfigurationSnapshot createLabel(final Revision revision, final String groupId, final LabelDTO labelDTO) { + return optimisticLockingManager.configureFlow(revision, new ConfigurationRequest() { + @Override + public LabelDTO execute() { + // ensure id is set + if (StringUtils.isBlank(labelDTO.getId())) { + labelDTO.setId(UUID.randomUUID().toString()); + } - // ensure the proper revision before performing the update - checkRevision(revision); + // add the label + final Label label = labelDAO.createLabel(groupId, labelDTO); - // ensure id is set - if (StringUtils.isBlank(labelDTO.getId())) { - labelDTO.setId(UUID.randomUUID().toString()); - } - - // add the label - final Label label = labelDAO.createLabel(groupId, labelDTO); - - // update the revision and generate a response - final Revision updatedRevision = updateRevision(revision); - final ConfigurationSnapshot response = new ConfigurationSnapshot<>(updatedRevision.getVersion(), dtoFactory.createLabelDto(label)); - - // save the flow - controllerFacade.save(); - - return response; + // save the flow + controllerFacade.save(); + + return dtoFactory.createLabelDto(label); + } + }); } @Override - public ConfigurationSnapshot createFunnel(Revision revision, String groupId, FunnelDTO funnelDTO) { - // ensure the proper revision before performing the update - checkRevision(revision); + public ConfigurationSnapshot createFunnel(final Revision revision, final String groupId, final FunnelDTO funnelDTO) { + return optimisticLockingManager.configureFlow(revision, new ConfigurationRequest() { + @Override + public FunnelDTO execute() { + // ensure id is set + if (StringUtils.isBlank(funnelDTO.getId())) { + funnelDTO.setId(UUID.randomUUID().toString()); + } - // ensure id is set - if (StringUtils.isBlank(funnelDTO.getId())) { - funnelDTO.setId(UUID.randomUUID().toString()); - } + // add the label + final Funnel funnel = funnelDAO.createFunnel(groupId, funnelDTO); - // add the label - final Funnel funnel = funnelDAO.createFunnel(groupId, funnelDTO); - - // update the revision and generate a response - final Revision updatedRevision = updateRevision(revision); - final ConfigurationSnapshot response = new ConfigurationSnapshot<>(updatedRevision.getVersion(), dtoFactory.createFunnelDto(funnel)); - - // save the flow - controllerFacade.save(); - - return response; + // save the flow + controllerFacade.save(); + + return dtoFactory.createFunnelDto(funnel); + } + }); } private void validateSnippetContents(final FlowSnippetDTO flowSnippet, final String groupId) { @@ -1008,139 +955,129 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade { } @Override - public ConfigurationSnapshot copySnippet(Revision revision, String groupId, String snippetId, Double originX, Double originY) { - // ensure the proper revision before performing the update - checkRevision(revision); + public ConfigurationSnapshot copySnippet(final Revision revision, final String groupId, final String snippetId, final Double originX, final Double originY) { + return optimisticLockingManager.configureFlow(revision, new ConfigurationRequest() { + @Override + public FlowSnippetDTO execute() { + String id = snippetId; + + // ensure id is set + if (StringUtils.isBlank(id)) { + id = UUID.randomUUID().toString(); + } - // ensure id is set - if (StringUtils.isBlank(snippetId)) { - snippetId = UUID.randomUUID().toString(); - } + // create the new snippet + FlowSnippetDTO flowSnippet = snippetDAO.copySnippet(groupId, id, originX, originY); - // create the new snippet - FlowSnippetDTO flowSnippet = snippetDAO.copySnippet(groupId, snippetId, originX, originY); + // validate the new snippet + validateSnippetContents(flowSnippet, groupId); - // validate the new snippet - validateSnippetContents(flowSnippet, groupId); - - // update the revision and generate a response - final Revision updatedRevision = updateRevision(revision); - final ConfigurationSnapshot response = new ConfigurationSnapshot<>(updatedRevision.getVersion(), flowSnippet); - - // save the flow - controllerFacade.save(); - - return response; + // save the flow + controllerFacade.save(); + + return flowSnippet; + } + }); } @Override public ConfigurationSnapshot createSnippet(final Revision revision, final SnippetDTO snippetDTO) { - // ensure the proper revision before performing the update - checkRevision(revision); + return optimisticLockingManager.configureFlow(revision, new ConfigurationRequest() { + @Override + public SnippetDTO execute() { + // ensure id is set + if (StringUtils.isBlank(snippetDTO.getId())) { + snippetDTO.setId(UUID.randomUUID().toString()); + } - // ensure id is set - if (StringUtils.isBlank(snippetDTO.getId())) { - snippetDTO.setId(UUID.randomUUID().toString()); - } - - // add the snippet - final Snippet snippet = snippetDAO.createSnippet(snippetDTO); - final SnippetDTO responseSnippetDTO = dtoFactory.createSnippetDto(snippet); - responseSnippetDTO.setContents(snippetUtils.populateFlowSnippet(snippet, false)); - - // create the response - final Revision updatedRevision = updateRevision(revision); - final ConfigurationSnapshot response = new ConfigurationSnapshot<>(updatedRevision.getVersion(), responseSnippetDTO); - - return response; + // add the snippet + final Snippet snippet = snippetDAO.createSnippet(snippetDTO); + final SnippetDTO responseSnippetDTO = dtoFactory.createSnippetDto(snippet); + responseSnippetDTO.setContents(snippetUtils.populateFlowSnippet(snippet, false)); + + return responseSnippetDTO; + } + }); } @Override - public ConfigurationSnapshot createInputPort(Revision revision, String groupId, PortDTO inputPortDTO) { - // ensure the proper revision before performing the update - checkRevision(revision); + public ConfigurationSnapshot createInputPort(final Revision revision, final String groupId, final PortDTO inputPortDTO) { + return optimisticLockingManager.configureFlow(revision, new ConfigurationRequest() { + @Override + public PortDTO execute() { + // ensure id is set + if (StringUtils.isBlank(inputPortDTO.getId())) { + inputPortDTO.setId(UUID.randomUUID().toString()); + } - // ensure id is set - if (StringUtils.isBlank(inputPortDTO.getId())) { - inputPortDTO.setId(UUID.randomUUID().toString()); - } + final Port inputPort = inputPortDAO.createPort(groupId, inputPortDTO); - final Port inputPort = inputPortDAO.createPort(groupId, inputPortDTO); - - // update the revision and generate a response - final Revision updatedRevision = updateRevision(revision); - final ConfigurationSnapshot response = new ConfigurationSnapshot<>(updatedRevision.getVersion(), dtoFactory.createPortDto(inputPort)); - - // save the flow - controllerFacade.save(); - - return response; + // save the flow + controllerFacade.save(); + + return dtoFactory.createPortDto(inputPort); + } + }); } @Override - public ConfigurationSnapshot createOutputPort(Revision revision, String groupId, PortDTO outputPortDTO) { - // ensure the proper revision before performing the update - checkRevision(revision); + public ConfigurationSnapshot createOutputPort(final Revision revision, final String groupId, final PortDTO outputPortDTO) { + return optimisticLockingManager.configureFlow(revision, new ConfigurationRequest() { + @Override + public PortDTO execute() { + // ensure id is set + if (StringUtils.isBlank(outputPortDTO.getId())) { + outputPortDTO.setId(UUID.randomUUID().toString()); + } - // ensure id is set - if (StringUtils.isBlank(outputPortDTO.getId())) { - outputPortDTO.setId(UUID.randomUUID().toString()); - } + final Port outputPort = outputPortDAO.createPort(groupId, outputPortDTO); - final Port outputPort = outputPortDAO.createPort(groupId, outputPortDTO); - - // update the revision and generate a response - final Revision updatedRevision = updateRevision(revision); - final ConfigurationSnapshot response = new ConfigurationSnapshot<>(updatedRevision.getVersion(), dtoFactory.createPortDto(outputPort)); - - // save the flow - controllerFacade.save(); - - return response; + // save the flow + controllerFacade.save(); + + return dtoFactory.createPortDto(outputPort); + } + }); } @Override - public ConfigurationSnapshot createProcessGroup(String parentGroupId, Revision revision, ProcessGroupDTO processGroupDTO) { - // ensure the proper revision before performing the update - checkRevision(revision); + public ConfigurationSnapshot createProcessGroup(final String parentGroupId, final Revision revision, final ProcessGroupDTO processGroupDTO) { + return optimisticLockingManager.configureFlow(revision, new ConfigurationRequest() { + @Override + public ProcessGroupDTO execute() { + // ensure id is set + if (StringUtils.isBlank(processGroupDTO.getId())) { + processGroupDTO.setId(UUID.randomUUID().toString()); + } - // ensure id is set - if (StringUtils.isBlank(processGroupDTO.getId())) { - processGroupDTO.setId(UUID.randomUUID().toString()); - } + final ProcessGroup processGroup = processGroupDAO.createProcessGroup(parentGroupId, processGroupDTO); - final ProcessGroup processGroup = processGroupDAO.createProcessGroup(parentGroupId, processGroupDTO); - - // update the revision and generate a response - final Revision updatedRevision = updateRevision(revision); - final ConfigurationSnapshot response = new ConfigurationSnapshot<>(updatedRevision.getVersion(), dtoFactory.createProcessGroupDto(processGroup)); - - // save the flow - controllerFacade.save(); - - return response; + // save the flow + controllerFacade.save(); + + return dtoFactory.createProcessGroupDto(processGroup); + } + }); } @Override - public ConfigurationSnapshot createRemoteProcessGroup(Revision revision, String groupId, RemoteProcessGroupDTO remoteProcessGroupDTO) { - // ensure the proper revision before performing the update - checkRevision(revision); + public ConfigurationSnapshot createRemoteProcessGroup(final Revision revision, final String groupId, final RemoteProcessGroupDTO remoteProcessGroupDTO) { + return optimisticLockingManager.configureFlow(revision, new ConfigurationRequest() { + @Override + public RemoteProcessGroupDTO execute() { + // ensure id is set + if (StringUtils.isBlank(remoteProcessGroupDTO.getId())) { + remoteProcessGroupDTO.setId(UUID.randomUUID().toString()); + } - // ensure id is set - if (StringUtils.isBlank(remoteProcessGroupDTO.getId())) { - remoteProcessGroupDTO.setId(UUID.randomUUID().toString()); - } + final RemoteProcessGroup remoteProcessGroup = remoteProcessGroupDAO.createRemoteProcessGroup(groupId, remoteProcessGroupDTO); - final RemoteProcessGroup remoteProcessGroup = remoteProcessGroupDAO.createRemoteProcessGroup(groupId, remoteProcessGroupDTO); - - // update the revision and generate a response - final Revision updatedRevision = updateRevision(revision); - final ConfigurationSnapshot response = new ConfigurationSnapshot<>(updatedRevision.getVersion(), dtoFactory.createRemoteProcessGroupDto(remoteProcessGroup)); - - // save the flow - controllerFacade.save(); - - return response; + // save the flow + controllerFacade.save(); + + return dtoFactory.createRemoteProcessGroupDto(remoteProcessGroup); + } + }); } @Override @@ -1186,74 +1123,217 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade { } @Override - public ConfigurationSnapshot createTemplateInstance(Revision revision, String groupId, Double originX, Double originY, String templateId) { - // ensure the proper revision before performing the update - checkRevision(revision); + public ConfigurationSnapshot createTemplateInstance(final Revision revision, final String groupId, final Double originX, final Double originY, final String templateId) { + return optimisticLockingManager.configureFlow(revision, new ConfigurationRequest() { + @Override + public FlowSnippetDTO execute() { + // instantiate the template - there is no need to make another copy of the flow snippet since the actual template + // was copied and this dto is only used to instantiate it's components (which as already completed) + FlowSnippetDTO flowSnippet = templateDAO.instantiateTemplate(groupId, originX, originY, templateId); - // instantiate the template - there is no need to make another copy of the flow snippet since the actual template - // was copied and this dto is only used to instantiate it's components (which as already completed) - FlowSnippetDTO flowSnippet = templateDAO.instantiateTemplate(groupId, originX, originY, templateId); + // validate the new snippet + validateSnippetContents(flowSnippet, groupId); - // validate the new snippet - validateSnippetContents(flowSnippet, groupId); - - // update the revision and generate a response - final Revision updatedRevision = updateRevision(revision); - final ConfigurationSnapshot response = new ConfigurationSnapshot<>(updatedRevision.getVersion(), flowSnippet); - - // save the flow - controllerFacade.save(); - - return response; + // save the flow + controllerFacade.save(); + + return flowSnippet; + } + }); } @Override - public ConfigurationSnapshot createArchive(Revision revision) { - // ensure the proper revision before performing the update - checkRevision(revision); - - // create the archive - controllerFacade.createArchive(); - - // update the revision and generate a response - final Revision updatedRevision = updateRevision(revision); - final ConfigurationSnapshot response = new ConfigurationSnapshot<>(updatedRevision.getVersion()); - return response; + public ConfigurationSnapshot createArchive(final Revision revision) { + return optimisticLockingManager.configureFlow(revision, new ConfigurationRequest() { + @Override + public Void execute() { + // create the archive + controllerFacade.createArchive(); + return null; + } + }); } @Override - public ConfigurationSnapshot setProcessorAnnotationData(Revision revision, String processorId, String annotationData) { - // ensure the proper revision before performing the update - checkRevision(revision); + public ConfigurationSnapshot setProcessorAnnotationData(final Revision revision, final String processorId, final String annotationData) { + return optimisticLockingManager.configureFlow(revision, new ConfigurationRequest() { + @Override + public ProcessorDTO execute() { + // create the processor config + final ProcessorConfigDTO config = new ProcessorConfigDTO(); + config.setAnnotationData(annotationData); - // create the processor config - final ProcessorConfigDTO config = new ProcessorConfigDTO(); - config.setAnnotationData(annotationData); + // create the processor dto + final ProcessorDTO processorDTO = new ProcessorDTO(); + processorDTO.setId(processorId); + processorDTO.setConfig(config); - // create the processor dto - final ProcessorDTO processorDTO = new ProcessorDTO(); - processorDTO.setId(processorId); - processorDTO.setConfig(config); + // get the parent group id for the specified processor + String groupId = controllerFacade.findProcessGroupIdForProcessor(processorId); - // get the parent group id for the specified processor - String groupId = controllerFacade.findProcessGroupIdForProcessor(processorId); + // ensure the parent group id was found + if (groupId == null) { + throw new ResourceNotFoundException(String.format("Unable to locate Processor with id '%s'.", processorId)); + } - // ensure the parent group id was found - if (groupId == null) { - throw new ResourceNotFoundException(String.format("Unable to locate Processor with id '%s'.", processorId)); + // update the processor configuration + ProcessorNode processor = processorDAO.updateProcessor(groupId, processorDTO); + + // save the flow + controllerFacade.save(); + + return dtoFactory.createProcessorDto(processor); + } + }); + } + + @Override + public ConfigurationSnapshot createControllerService(final Revision revision, final ControllerServiceDTO controllerServiceDTO) { + return optimisticLockingManager.configureFlow(revision, new ConfigurationRequest() { + @Override + public ControllerServiceDTO execute() { + // ensure id is set + if (StringUtils.isBlank(controllerServiceDTO.getId())) { + controllerServiceDTO.setId(UUID.randomUUID().toString()); + } + + // create the controller service + final ControllerServiceNode controllerService = controllerServiceDAO.createControllerService(controllerServiceDTO); + + // save the update + if (properties.isClusterManager()) { + clusterManager.saveControllerServices(); + } else { + controllerFacade.save(); + } + + return dtoFactory.createControllerServiceDto(controllerService); + } + }); + } + + @Override + public ConfigurationSnapshot updateControllerService(final Revision revision, final ControllerServiceDTO controllerServiceDTO) { + // if controller service does not exist, then create new controller service + if (controllerServiceDAO.hasControllerService(controllerServiceDTO.getId()) == false) { + return createControllerService(revision, controllerServiceDTO); } + + return optimisticLockingManager.configureFlow(revision, new ConfigurationRequest() { + @Override + public ControllerServiceDTO execute() { + final ControllerServiceNode controllerService = controllerServiceDAO.updateControllerService(controllerServiceDTO); - // update the processor configuration - ProcessorNode processor = processorDAO.updateProcessor(groupId, processorDTO); + // save the update + if (properties.isClusterManager()) { + clusterManager.saveControllerServices(); + } else { + controllerFacade.save(); + } - // update the revision and generate a response - final Revision updatedRevision = updateRevision(revision); - final ConfigurationSnapshot response = new ConfigurationSnapshot<>(updatedRevision.getVersion(), dtoFactory.createProcessorDto(processor)); + return dtoFactory.createControllerServiceDto(controllerService); + } + }); + } - // save the flow - controllerFacade.save(); + @Override + public ConfigurationSnapshot> updateControllerServiceReferencingComponents(final Revision revision, final String controllerServiceId, final org.apache.nifi.controller.ScheduledState scheduledState, final org.apache.nifi.controller.service.ControllerServiceState controllerServiceState) { + return optimisticLockingManager.configureFlow(revision, new ConfigurationRequest>() { + @Override + public Set execute() { + final ControllerServiceReference reference = controllerServiceDAO.updateControllerServiceReferencingComponents(controllerServiceId, scheduledState, controllerServiceState); + return dtoFactory.createControllerServiceReferencingComponentsDto(reference); + } + }); + } - return response; + @Override + public ConfigurationSnapshot deleteControllerService(final Revision revision, final String controllerServiceId) { + return optimisticLockingManager.configureFlow(revision, new ConfigurationRequest() { + @Override + public Void execute() { + // delete the label + controllerServiceDAO.deleteControllerService(controllerServiceId); + + // save the update + if (properties.isClusterManager()) { + clusterManager.saveControllerServices(); + } else { + controllerFacade.save(); + } + + return null; + } + }); + } + + @Override + public ConfigurationSnapshot createReportingTask(final Revision revision, final ReportingTaskDTO reportingTaskDTO) { + return optimisticLockingManager.configureFlow(revision, new ConfigurationRequest() { + @Override + public ReportingTaskDTO execute() { + // ensure id is set + if (StringUtils.isBlank(reportingTaskDTO.getId())) { + reportingTaskDTO.setId(UUID.randomUUID().toString()); + } + + // create the reporting + final ReportingTaskNode reportingTask = reportingTaskDAO.createReportingTask(reportingTaskDTO); + + // save the update + if (properties.isClusterManager()) { + clusterManager.saveReportingTasks(); + } else { + controllerFacade.save(); + } + + return dtoFactory.createReportingTaskDto(reportingTask); + } + }); + } + + @Override + public ConfigurationSnapshot updateReportingTask(final Revision revision, final ReportingTaskDTO reportingTaskDTO) { + // if reporting task does not exist, then create new reporting task + if (reportingTaskDAO.hasReportingTask(reportingTaskDTO.getId()) == false) { + return createReportingTask(revision, reportingTaskDTO); + } + + return optimisticLockingManager.configureFlow(revision, new ConfigurationRequest() { + @Override + public ReportingTaskDTO execute() { + final ReportingTaskNode reportingTask = reportingTaskDAO.updateReportingTask(reportingTaskDTO); + + // save the update + if (properties.isClusterManager()) { + clusterManager.saveReportingTasks(); + } else { + controllerFacade.save(); + } + + return dtoFactory.createReportingTaskDto(reportingTask); + } + }); + } + + @Override + public ConfigurationSnapshot deleteReportingTask(final Revision revision, final String reportingTaskId) { + return optimisticLockingManager.configureFlow(revision, new ConfigurationRequest() { + @Override + public Void execute() { + // delete the label + reportingTaskDAO.deleteReportingTask(reportingTaskId); + + // save the update + if (properties.isClusterManager()) { + clusterManager.saveReportingTasks(); + } else { + controllerFacade.save(); + } + + return null; + } + }); } @Override @@ -1408,9 +1488,10 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade { // ----------------------------------------- // Read Operations // ----------------------------------------- + @Override public RevisionDTO getRevision() { - return dtoFactory.createRevisionDTO(optimisticLockingManager.getRevision()); + return dtoFactory.createRevisionDTO(optimisticLockingManager.getLastModification()); } @Override @@ -1636,6 +1717,16 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade { return controllerFacade.getFlowFileProcessorTypes(); } + @Override + public Set getControllerServiceTypes() { + return controllerFacade.getControllerServiceTypes(); + } + + @Override + public Set getReportingTaskTypes() { + return controllerFacade.getReportingTaskTypes(); + } + @Override public ProcessorDTO getProcessor(String groupId, String id) { final ProcessorNode processor = processorDAO.getProcessor(groupId, id); @@ -1643,6 +1734,19 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade { return processorDto; } + @Override + public PropertyDescriptorDTO getProcessorPropertyDescriptor(String groupId, String id, String property) { + final ProcessorNode processor = processorDAO.getProcessor(groupId, id); + PropertyDescriptor descriptor = processor.getPropertyDescriptor(property); + + // return an invalid descriptor if the processor doesn't suppor this property + if (descriptor == null) { + descriptor = new PropertyDescriptor.Builder().name(property).addValidator(Validator.INVALID).dynamic(true).build(); + } + + return dtoFactory.createPropertyDescriptorDto(descriptor); + } + @Override public StatusHistoryDTO getProcessorStatusHistory(String groupId, String id) { return controllerFacade.getProcessorStatusHistory(groupId, id); @@ -1823,6 +1927,7 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade { final Date now = new Date(); controllerConfig.setTimeOffset(TimeZone.getDefault().getOffset(now.getTime())); + controllerConfig.setCurrentTime(now); // determine the site to site configuration if (isClustered()) { @@ -1929,11 +2034,71 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade { @Override public ConfigurationSnapshot getProcessGroup(String groupId, final boolean recurse) { ProcessGroup processGroup = processGroupDAO.getProcessGroup(groupId); - Long version = optimisticLockingManager.getRevision().getVersion(); - ConfigurationSnapshot response = new ConfigurationSnapshot<>(version, dtoFactory.createProcessGroupDto(processGroup, recurse)); + Revision revision = optimisticLockingManager.getLastModification().getRevision(); + ConfigurationSnapshot response = new ConfigurationSnapshot<>(revision.getVersion(), dtoFactory.createProcessGroupDto(processGroup, recurse)); return response; } + @Override + public Set getControllerServices() { + final Set controllerServiceDtos = new LinkedHashSet<>(); + for (ControllerServiceNode controllerService : controllerServiceDAO.getControllerServices()) { + controllerServiceDtos.add(dtoFactory.createControllerServiceDto(controllerService)); + } + return controllerServiceDtos; + } + + @Override + public ControllerServiceDTO getControllerService(String controllerServiceId) { + return dtoFactory.createControllerServiceDto(controllerServiceDAO.getControllerService(controllerServiceId)); + } + + @Override + public PropertyDescriptorDTO getControllerServicePropertyDescriptor(String id, String property) { + final ControllerServiceNode controllerService = controllerServiceDAO.getControllerService(id); + PropertyDescriptor descriptor = controllerService.getControllerServiceImplementation().getPropertyDescriptor(property); + + // return an invalid descriptor if the controller service doesn't support this property + if (descriptor == null) { + descriptor = new PropertyDescriptor.Builder().name(property).addValidator(Validator.INVALID).dynamic(true).build(); + } + + return dtoFactory.createPropertyDescriptorDto(descriptor); + } + + @Override + public Set getControllerServiceReferencingComponents(String controllerServiceId) { + final ControllerServiceNode service = controllerServiceDAO.getControllerService(controllerServiceId); + return dtoFactory.createControllerServiceReferencingComponentsDto(service.getReferences()); + } + + @Override + public Set getReportingTasks() { + final Set reportingTaskDtos = new LinkedHashSet<>(); + for (ReportingTaskNode reportingTask : reportingTaskDAO.getReportingTasks()) { + reportingTaskDtos.add(dtoFactory.createReportingTaskDto(reportingTask)); + } + return reportingTaskDtos; + } + + @Override + public ReportingTaskDTO getReportingTask(String reportingTaskId) { + return dtoFactory.createReportingTaskDto(reportingTaskDAO.getReportingTask(reportingTaskId)); + } + + @Override + public PropertyDescriptorDTO getReportingTaskPropertyDescriptor(String id, String property) { + final ReportingTaskNode reportingTask = reportingTaskDAO.getReportingTask(id); + PropertyDescriptor descriptor = reportingTask.getReportingTask().getPropertyDescriptor(property); + + // return an invalid descriptor if the reporting task doesn't support this property + if (descriptor == null) { + descriptor = new PropertyDescriptor.Builder().name(property).addValidator(Validator.INVALID).dynamic(true).build(); + } + + return dtoFactory.createPropertyDescriptorDto(descriptor); + } + @Override public StatusHistoryDTO getProcessGroupStatusHistory(String groupId) { return controllerFacade.getProcessGroupStatusHistory(groupId); @@ -1974,9 +2139,9 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade { } @Override - public ProcessorHistoryDTO getProcessorHistory(String processorId) { + public ComponentHistoryDTO getComponentHistory(String componentId) { final Map propertyHistoryDtos = new LinkedHashMap<>(); - final Map> propertyHistory = auditService.getPreviousValues(processorId); + final Map> propertyHistory = auditService.getPreviousValues(componentId); for (final Map.Entry> entry : propertyHistory.entrySet()) { final List previousValueDtos = new ArrayList<>(); @@ -1996,8 +2161,8 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade { } } - final ProcessorHistoryDTO history = new ProcessorHistoryDTO(); - history.setProcessorId(processorId); + final ComponentHistoryDTO history = new ComponentHistoryDTO(); + history.setComponentId(componentId); history.setPropertyHistory(propertyHistoryDtos); return history; @@ -2718,6 +2883,14 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade { this.processGroupDAO = processGroupDAO; } + public void setControllerServiceDAO(ControllerServiceDAO controllerServiceDAO) { + this.controllerServiceDAO = controllerServiceDAO; + } + + public void setReportingTaskDAO(ReportingTaskDAO reportingTaskDAO) { + this.reportingTaskDAO = reportingTaskDAO; + } + public void setTemplateDAO(TemplateDAO templateDAO) { this.templateDAO = templateDAO; } diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiWebConfigurationContext.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiWebConfigurationContext.java new file mode 100644 index 0000000000..8d51a584f5 --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiWebConfigurationContext.java @@ -0,0 +1,736 @@ +/* + * 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.web; + +import java.io.Serializable; +import java.io.UnsupportedEncodingException; +import java.net.URI; +import java.net.URISyntaxException; +import java.net.URLEncoder; +import java.util.Collection; +import java.util.Date; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Objects; + +import javax.ws.rs.HttpMethod; +import javax.ws.rs.core.MultivaluedMap; +import javax.ws.rs.core.Response; + +import org.apache.nifi.action.Action; +import org.apache.nifi.action.Component; +import org.apache.nifi.action.Operation; +import org.apache.nifi.action.component.details.ExtensionDetails; +import org.apache.nifi.action.details.ConfigureDetails; +import org.apache.nifi.admin.service.AuditService; +import org.apache.nifi.cluster.manager.NodeResponse; +import org.apache.nifi.cluster.manager.impl.WebClusterManager; +import org.apache.nifi.controller.ControllerService; +import org.apache.nifi.web.security.user.NiFiUserDetails; +import org.apache.nifi.web.security.user.NiFiUserUtils; +import org.apache.nifi.user.NiFiUser; +import org.apache.nifi.util.NiFiProperties; +import org.apache.nifi.web.api.dto.ProcessorConfigDTO; +import org.apache.nifi.web.api.dto.ProcessorDTO; +import org.apache.nifi.web.api.dto.RevisionDTO; +import org.apache.nifi.web.api.entity.ProcessorEntity; +import org.apache.nifi.web.util.WebUtils; + +import org.apache.commons.lang3.StringUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.springframework.security.access.prepost.PreAuthorize; +import org.springframework.security.core.Authentication; +import org.springframework.security.core.context.SecurityContextHolder; + +import com.sun.jersey.core.util.MultivaluedMapImpl; +import org.apache.nifi.controller.ControllerServiceLookup; +import org.apache.nifi.controller.reporting.ReportingTaskProvider; +import org.apache.nifi.web.api.dto.ControllerServiceDTO; +import org.apache.nifi.web.api.dto.ReportingTaskDTO; +import org.apache.nifi.web.api.entity.ControllerServiceEntity; +import org.apache.nifi.web.api.entity.ReportingTaskEntity; +import org.apache.nifi.web.util.ClientResponseUtils; + +/** + * Implements the NiFiWebConfigurationContext interface to support a context in both + * standalone and clustered environments. + */ +public class StandardNiFiWebConfigurationContext implements NiFiWebConfigurationContext { + + private static final Logger logger = LoggerFactory.getLogger(StandardNiFiWebConfigurationContext.class); + public static final String CLIENT_ID_PARAM = "clientId"; + public static final String REVISION_PARAM = "revision"; + public static final String VERBOSE_PARAM = "verbose"; + + private NiFiProperties properties; + private NiFiServiceFacade serviceFacade; + private WebClusterManager clusterManager; + private ControllerServiceLookup controllerServiceLookup; + private ReportingTaskProvider reportingTaskProvider; + private AuditService auditService; + + @Override + public ControllerService getControllerService(String serviceIdentifier) { + return controllerServiceLookup.getControllerService(serviceIdentifier); + } + + @Override + @PreAuthorize("hasAnyRole('ROLE_DFM')") + public void saveActions(final NiFiWebRequestContext requestContext, final Collection configurationActions) { + Objects.requireNonNull(configurationActions, "Actions cannot be null."); + + // ensure the path could be + if (requestContext.getExtensionType() == null) { + throw new IllegalArgumentException("The UI extension type must be specified."); + } + + Component componentType = null; + switch (requestContext.getExtensionType()) { + case ProcessorConfiguration: + componentType = Component.Processor; + break; + case ControllerServiceConfiguration: + componentType = Component.ControllerService; + break; + case ReportingTaskConfiguration: + componentType = Component.ReportingTask; + break; + } + + if (componentType == null) { + throw new IllegalArgumentException("UI extension type must support Processor, ControllerService, or ReportingTask configuration."); + } + + // - when running standalone or cluster ncm - actions from custom UIs are stored locally + // - clustered nodes do not serve custom UIs directly to users so they should never be invoking this method + final Date now = new Date(); + final Collection actions = new HashSet<>(configurationActions.size()); + for (final ConfigurationAction configurationAction : configurationActions) { + final ExtensionDetails extensionDetails = new ExtensionDetails(); + extensionDetails.setType(configurationAction.getType()); + + final ConfigureDetails configureDetails = new ConfigureDetails(); + configureDetails.setName(configurationAction.getName()); + configureDetails.setPreviousValue(configurationAction.getPreviousValue()); + configureDetails.setValue(configurationAction.getValue()); + + final Action action = new Action(); + action.setTimestamp(now); + action.setSourceId(configurationAction.getId()); + action.setSourceName(configurationAction.getName()); + action.setSourceType(componentType); + action.setOperation(Operation.Configure); + action.setUserDn(getCurrentUserDn()); + action.setUserName(getCurrentUserName()); + action.setComponentDetails(extensionDetails); + action.setActionDetails(configureDetails); + actions.add(action); + } + + if (!actions.isEmpty()) { + try { + // record the operations + auditService.addActions(actions); + } catch (Throwable t) { + logger.warn("Unable to record actions: " + t.getMessage()); + if (logger.isDebugEnabled()) { + logger.warn(StringUtils.EMPTY, t); + } + } + } + } + + @Override + public String getCurrentUserDn() { + String userDn = NiFiUser.ANONYMOUS_USER_DN; + + final NiFiUser user = NiFiUserUtils.getNiFiUser(); + if (user != null) { + userDn = user.getDn(); + } + + return userDn; + } + + @Override + public String getCurrentUserName() { + String userName = NiFiUser.ANONYMOUS_USER_DN; + + final NiFiUser user = NiFiUserUtils.getNiFiUser(); + if (user != null) { + userName = user.getUserName(); + } + + return userName; + } + + @Override + public ComponentDetails getComponentDetails(final NiFiWebRequestContext requestContext) throws ResourceNotFoundException, ClusterRequestException { + final String id = requestContext.getId(); + + if (StringUtils.isBlank(id)) { + throw new ResourceNotFoundException(String.format("Configuration request context config did not have a component ID.")); + } + + // ensure the path could be + if (requestContext.getExtensionType() == null) { + throw new IllegalArgumentException("The UI extension type must be specified."); + } + + // get the component facade for interacting directly with that type of object + ComponentFacade componentFacade = null; + switch (requestContext.getExtensionType()) { + case ProcessorConfiguration: + componentFacade = new ProcessorFacade(); + break; + case ControllerServiceConfiguration: + componentFacade = new ControllerServiceFacade(); + break; + case ReportingTaskConfiguration: + componentFacade = new ReportingTaskFacade(); + break; + } + + if (componentFacade == null) { + throw new IllegalArgumentException("UI extension type must support Processor, ControllerService, or ReportingTask configuration."); + } + + return componentFacade.getComponentDetails(requestContext); + } + + @Override + @PreAuthorize("hasAnyRole('ROLE_DFM')") + public ComponentDetails setAnnotationData(final NiFiWebConfigurationRequestContext requestContext, final String annotationData) + throws ResourceNotFoundException, InvalidRevisionException, ClusterRequestException { + + final String id = requestContext.getId(); + + if (StringUtils.isBlank(id)) { + throw new ResourceNotFoundException(String.format("Configuration request context did not have a component ID.")); + } + + // ensure the path could be + if (requestContext.getExtensionType() == null) { + throw new IllegalArgumentException("The UI extension type must be specified."); + } + + // get the component facade for interacting directly with that type of object + ComponentFacade componentFacade = null; + switch (requestContext.getExtensionType()) { + case ProcessorConfiguration: + componentFacade = new ProcessorFacade(); + break; + case ControllerServiceConfiguration: + componentFacade = new ControllerServiceFacade(); + break; + case ReportingTaskConfiguration: + componentFacade = new ReportingTaskFacade(); + break; + } + + if (componentFacade == null) { + throw new IllegalArgumentException("UI extension type must support Processor, ControllerService, or ReportingTask configuration."); + } + + return componentFacade.setAnnotationData(requestContext, annotationData); + } + + /** + * Facade over accessing different types of NiFi components. + */ + private interface ComponentFacade { + /** + * Gets the component details using the specified request context. + * + * @param requestContext + * @return + */ + ComponentDetails getComponentDetails(NiFiWebRequestContext requestContext); + + /** + * Sets the annotation data using the specified request context. + * + * @param requestContext + * @param annotationData + * @return + */ + ComponentDetails setAnnotationData(NiFiWebConfigurationRequestContext requestContext, String annotationData); + } + + /** + * Interprets the request/response with the underlying Processor model. + */ + private class ProcessorFacade implements ComponentFacade { + @Override + public ComponentDetails getComponentDetails(final NiFiWebRequestContext requestContext) { + final String id = requestContext.getId(); + + final ProcessorDTO processor; + if (properties.isClusterManager()) { + // create the request URL + URI requestUrl; + try { + String path = "/nifi-api/cluster/processors/" + URLEncoder.encode(id, "UTF-8"); + requestUrl = new URI(requestContext.getScheme(), null, "localhost", 0, path, null, null); + } catch (final URISyntaxException | UnsupportedEncodingException use) { + throw new ClusterRequestException(use); + } + + // set the request parameters + MultivaluedMap parameters = new MultivaluedMapImpl(); + parameters.add(VERBOSE_PARAM, "true"); + + // replicate request + NodeResponse nodeResponse = clusterManager.applyRequest(HttpMethod.GET, requestUrl, parameters, getHeaders(requestContext)); + + // check for issues replicating request + checkResponse(nodeResponse, id); + + // return processor + ProcessorEntity entity = (ProcessorEntity) nodeResponse.getUpdatedEntity(); + if (entity == null) { + entity = nodeResponse.getClientResponse().getEntity(ProcessorEntity.class); + } + processor = entity.getProcessor(); + } else { + processor = serviceFacade.getProcessor(id); + } + + // return the processor info + return getComponentConfiguration(processor); + } + + @Override + public ComponentDetails setAnnotationData(final NiFiWebConfigurationRequestContext requestContext, final String annotationData) { + final Revision revision = requestContext.getRevision(); + final String id = requestContext.getId(); + + final ProcessorDTO processor; + if (properties.isClusterManager()) { + // create the request URL + URI requestUrl; + try { + String path = "/nifi-api/cluster/processors/" + URLEncoder.encode(id, "UTF-8"); + requestUrl = new URI(requestContext.getScheme(), null, "localhost", 0, path, null, null); + } catch (final URISyntaxException | UnsupportedEncodingException use) { + throw new ClusterRequestException(use); + } + + // create the revision + RevisionDTO revisionDto = new RevisionDTO(); + revisionDto.setClientId(revision.getClientId()); + revisionDto.setVersion(revision.getVersion()); + + // create the processor entity + ProcessorEntity processorEntity = new ProcessorEntity(); + processorEntity.setRevision(revisionDto); + + // create the processor dto + ProcessorDTO processorDto = new ProcessorDTO(); + processorEntity.setProcessor(processorDto); + processorDto.setId(id); + + // create the processor configuration with the given annotation data + ProcessorConfigDTO configDto = new ProcessorConfigDTO(); + processorDto.setConfig(configDto); + configDto.setAnnotationData(annotationData); + + // set the content type to json + final Map headers = getHeaders(requestContext); + headers.put("Content-Type", "application/json"); + + // replicate request + NodeResponse nodeResponse = clusterManager.applyRequest(HttpMethod.PUT, requestUrl, processorEntity, headers); + + // check for issues replicating request + checkResponse(nodeResponse, id); + + // return processor + ProcessorEntity entity = (ProcessorEntity) nodeResponse.getUpdatedEntity(); + if (entity == null) { + entity = nodeResponse.getClientResponse().getEntity(ProcessorEntity.class); + } + processor = entity.getProcessor(); + } else { + final ConfigurationSnapshot response = serviceFacade.setProcessorAnnotationData(revision, id, annotationData); + processor = response.getConfiguration(); + } + + // return the processor info + return getComponentConfiguration(processor); + } + + private ComponentDetails getComponentConfiguration(final ProcessorDTO processor) { + final ProcessorConfigDTO processorConfig = processor.getConfig(); + return new ComponentDetails.Builder() + .id(processor.getId()) + .name(processor.getName()) + .type(processor.getType()) + .state(processor.getState()) + .annotationData(processorConfig.getAnnotationData()) + .properties(processorConfig.getProperties()) + .validateErrors(processor.getValidationErrors()).build(); + } + } + + /** + * Interprets the request/response with the underlying ControllerService model. + */ + private class ControllerServiceFacade implements ComponentFacade { + @Override + public ComponentDetails getComponentDetails(final NiFiWebRequestContext requestContext) { + final String id = requestContext.getId(); + final ControllerServiceDTO controllerService; + + // if the lookup has the service that means we are either a node or + // the ncm and the service is available there only + if (controllerServiceLookup.getControllerService(id) != null) { + controllerService = serviceFacade.getControllerService(id); + } else { + // if this is a standalone instance the service should have been found above... there should + // no cluster to replicate the request to + if (!properties.isClusterManager()) { + throw new ResourceNotFoundException(String.format("Controller service[%s] could not be found on this NiFi.", id)); + } + + // create the request URL + URI requestUrl; + try { + String path = "/nifi-api/controller/controller-services/node/" + URLEncoder.encode(id, "UTF-8"); + requestUrl = new URI(requestContext.getScheme(), null, "localhost", 0, path, null, null); + } catch (final URISyntaxException | UnsupportedEncodingException use) { + throw new ClusterRequestException(use); + } + + // set the request parameters + MultivaluedMap parameters = new MultivaluedMapImpl(); + + // replicate request + NodeResponse nodeResponse = clusterManager.applyRequest(HttpMethod.GET, requestUrl, parameters, getHeaders(requestContext)); + + // check for issues replicating request + checkResponse(nodeResponse, id); + + // return controller service + ControllerServiceEntity entity = (ControllerServiceEntity) nodeResponse.getUpdatedEntity(); + if (entity == null) { + entity = nodeResponse.getClientResponse().getEntity(ControllerServiceEntity.class); + } + controllerService = entity.getControllerService(); + } + + // return the controller service info + return getComponentConfiguration(controllerService); + } + + @Override + public ComponentDetails setAnnotationData(final NiFiWebConfigurationRequestContext requestContext, final String annotationData) { + final Revision revision = requestContext.getRevision(); + final String id = requestContext.getId(); + + final ControllerServiceDTO controllerService; + if (controllerServiceLookup.getControllerService(id) != null) { + final ControllerServiceDTO controllerServiceDto = new ControllerServiceDTO(); + controllerServiceDto.setId(id); + controllerServiceDto.setAnnotationData(annotationData); + + final ConfigurationSnapshot response = serviceFacade.updateControllerService(revision, controllerServiceDto); + controllerService = response.getConfiguration(); + } else { + // if this is a standalone instance the service should have been found above... there should + // no cluster to replicate the request to + if (!properties.isClusterManager()) { + throw new ResourceNotFoundException(String.format("Controller service[%s] could not be found on this NiFi.", id)); + } + + // since this PUT request can be interpreted as a request to create a controller service + // we need to be sure that this service exists on the node before the request is replicated. + // this is done by attempting to get the details. if the service doesn't exist it will + // throw a ResourceNotFoundException + getComponentDetails(requestContext); + + // create the request URL + URI requestUrl; + try { + String path = "/nifi-api/controller/controller-services/node/" + URLEncoder.encode(id, "UTF-8"); + requestUrl = new URI(requestContext.getScheme(), null, "localhost", 0, path, null, null); + } catch (final URISyntaxException | UnsupportedEncodingException use) { + throw new ClusterRequestException(use); + } + + // create the revision + RevisionDTO revisionDto = new RevisionDTO(); + revisionDto.setClientId(revision.getClientId()); + revisionDto.setVersion(revision.getVersion()); + + // create the controller service entity + ControllerServiceEntity controllerServiceEntity = new ControllerServiceEntity(); + controllerServiceEntity.setRevision(revisionDto); + + // create the controller service dto + ControllerServiceDTO controllerServiceDto = new ControllerServiceDTO(); + controllerServiceEntity.setControllerService(controllerServiceDto); + controllerServiceDto.setId(id); + controllerServiceDto.setAnnotationData(annotationData); + + // set the content type to json + final Map headers = getHeaders(requestContext); + headers.put("Content-Type", "application/json"); + + // replicate request + NodeResponse nodeResponse = clusterManager.applyRequest(HttpMethod.PUT, requestUrl, controllerServiceEntity, headers); + + // check for issues replicating request + checkResponse(nodeResponse, id); + + // return controller service + ControllerServiceEntity entity = (ControllerServiceEntity) nodeResponse.getUpdatedEntity(); + if (entity == null) { + entity = nodeResponse.getClientResponse().getEntity(ControllerServiceEntity.class); + } + controllerService = entity.getControllerService(); + } + + // return the controller service info + return getComponentConfiguration(controllerService); + } + + private ComponentDetails getComponentConfiguration(final ControllerServiceDTO controllerService) { + return new ComponentDetails.Builder() + .id(controllerService.getId()) + .name(controllerService.getName()) + .type(controllerService.getType()) + .state(controllerService.getState()) + .annotationData(controllerService.getAnnotationData()) + .properties(controllerService.getProperties()) + .validateErrors(controllerService.getValidationErrors()).build(); + } + } + + /** + * Interprets the request/response with the underlying ControllerService model. + */ + private class ReportingTaskFacade implements ComponentFacade { + @Override + public ComponentDetails getComponentDetails(final NiFiWebRequestContext requestContext) { + final String id = requestContext.getId(); + final ReportingTaskDTO reportingTask; + + // if the provider has the service that means we are either a node or + // the ncm and the service is available there only + if (reportingTaskProvider.getReportingTaskNode(id) != null) { + reportingTask = serviceFacade.getReportingTask(id); + } else { + // if this is a standalone instance the task should have been found above... there should + // no cluster to replicate the request to + if (!properties.isClusterManager()) { + throw new ResourceNotFoundException(String.format("Reporting task[%s] could not be found on this NiFi.", id)); + } + + // create the request URL + URI requestUrl; + try { + String path = "/nifi-api/controller/reporting-tasks/node/" + URLEncoder.encode(id, "UTF-8"); + requestUrl = new URI(requestContext.getScheme(), null, "localhost", 0, path, null, null); + } catch (final URISyntaxException | UnsupportedEncodingException use) { + throw new ClusterRequestException(use); + } + + // set the request parameters + MultivaluedMap parameters = new MultivaluedMapImpl(); + + // replicate request + NodeResponse nodeResponse = clusterManager.applyRequest(HttpMethod.GET, requestUrl, parameters, getHeaders(requestContext)); + + // check for issues replicating request + checkResponse(nodeResponse, id); + + // return reporting task + ReportingTaskEntity entity = (ReportingTaskEntity) nodeResponse.getUpdatedEntity(); + if (entity == null) { + entity = nodeResponse.getClientResponse().getEntity(ReportingTaskEntity.class); + } + reportingTask = entity.getReportingTask(); + } + + // return the reporting task info + return getComponentConfiguration(reportingTask); + } + + @Override + public ComponentDetails setAnnotationData(final NiFiWebConfigurationRequestContext requestContext, final String annotationData) { + final Revision revision = requestContext.getRevision(); + final String id = requestContext.getId(); + + final ReportingTaskDTO reportingTask; + if (reportingTaskProvider.getReportingTaskNode(id) != null) { + final ReportingTaskDTO reportingTaskDto = new ReportingTaskDTO(); + reportingTaskDto.setId(id); + reportingTaskDto.setAnnotationData(annotationData); + + final ConfigurationSnapshot response = serviceFacade.updateReportingTask(revision, reportingTaskDto); + reportingTask = response.getConfiguration(); + } else { + // if this is a standalone instance the task should have been found above... there should + // no cluster to replicate the request to + if (!properties.isClusterManager()) { + throw new ResourceNotFoundException(String.format("Reporting task[%s] could not be found on this NiFi.", id)); + } + + // since this PUT request can be interpreted as a request to create a reporting task + // we need to be sure that this task exists on the node before the request is replicated. + // this is done by attempting to get the details. if the service doesn't exist it will + // throw a ResourceNotFoundException + getComponentDetails(requestContext); + + // create the request URL + URI requestUrl; + try { + String path = "/nifi-api/controller/reporting-tasks/node/" + URLEncoder.encode(id, "UTF-8"); + requestUrl = new URI(requestContext.getScheme(), null, "localhost", 0, path, null, null); + } catch (final URISyntaxException | UnsupportedEncodingException use) { + throw new ClusterRequestException(use); + } + + // create the revision + RevisionDTO revisionDto = new RevisionDTO(); + revisionDto.setClientId(revision.getClientId()); + revisionDto.setVersion(revision.getVersion()); + + // create the reporting task entity + ReportingTaskEntity reportingTaskEntity = new ReportingTaskEntity(); + reportingTaskEntity.setRevision(revisionDto); + + // create the reporting task dto + ReportingTaskDTO reportingTaskDto = new ReportingTaskDTO(); + reportingTaskEntity.setReportingTask(reportingTaskDto); + reportingTaskDto.setId(id); + reportingTaskDto.setAnnotationData(annotationData); + + // set the content type to json + final Map headers = getHeaders(requestContext); + headers.put("Content-Type", "application/json"); + + // replicate request + NodeResponse nodeResponse = clusterManager.applyRequest(HttpMethod.PUT, requestUrl, reportingTaskEntity, headers); + + // check for issues replicating request + checkResponse(nodeResponse, id); + + // return reporting task + ReportingTaskEntity entity = (ReportingTaskEntity) nodeResponse.getUpdatedEntity(); + if (entity == null) { + entity = nodeResponse.getClientResponse().getEntity(ReportingTaskEntity.class); + } + reportingTask = entity.getReportingTask(); + } + + // return the processor info + return getComponentConfiguration(reportingTask); + } + + private ComponentDetails getComponentConfiguration(final ReportingTaskDTO reportingTask) { + return new ComponentDetails.Builder() + .id(reportingTask.getId()) + .name(reportingTask.getName()) + .type(reportingTask.getType()) + .state(reportingTask.getState()) + .annotationData(reportingTask.getAnnotationData()) + .properties(reportingTask.getProperties()) + .validateErrors(reportingTask.getValidationErrors()).build(); + } + } + + /** + * Gets the headers for the request to replicate to each node while + * clustered. + * + * @param config + * @return + */ + private Map getHeaders(final NiFiWebRequestContext config) { + final Map headers = new HashMap<>(); + headers.put("Accept", "application/json,application/xml"); + if (StringUtils.isNotBlank(config.getProxiedEntitiesChain())) { + headers.put("X-ProxiedEntitiesChain", config.getProxiedEntitiesChain()); + } + + // add the user's authorities (if any) to the headers + final Authentication authentication = SecurityContextHolder.getContext().getAuthentication(); + if (authentication != null) { + final Object userDetailsObj = authentication.getPrincipal(); + if (userDetailsObj instanceof NiFiUserDetails) { + // serialize user details object + final String hexEncodedUserDetails = WebUtils.serializeObjectToHex((Serializable) userDetailsObj); + + // put serialized user details in header + headers.put("X-ProxiedEntityUserDetails", hexEncodedUserDetails); + } + } + return headers; + } + + /** + * Checks the specified response and drains the stream appropriately. + * + * @param nodeResponse + * @param revision + * @param id + */ + private void checkResponse(final NodeResponse nodeResponse, final String id) { + if (nodeResponse.hasThrowable()) { + ClientResponseUtils.drainClientResponse(nodeResponse.getClientResponse()); + throw new ClusterRequestException(nodeResponse.getThrowable()); + } else if (nodeResponse.getClientResponse().getStatus() == Response.Status.CONFLICT.getStatusCode()) { + ClientResponseUtils.drainClientResponse(nodeResponse.getClientResponse()); + throw new InvalidRevisionException(String.format("Conflict: the flow may have been updated by another user.")); + } else if (nodeResponse.getClientResponse().getStatus() == Response.Status.NOT_FOUND.getStatusCode()) { + ClientResponseUtils.drainClientResponse(nodeResponse.getClientResponse()); + throw new ResourceNotFoundException("Unable to find component with id: " + id); + } else if (nodeResponse.getClientResponse().getStatus() != Response.Status.OK.getStatusCode()) { + ClientResponseUtils.drainClientResponse(nodeResponse.getClientResponse()); + throw new ClusterRequestException("Method resulted in an unsuccessful HTTP response code: " + nodeResponse.getClientResponse().getStatus()); + } + } + + public void setClusterManager(WebClusterManager clusterManager) { + this.clusterManager = clusterManager; + } + + public void setProperties(NiFiProperties properties) { + this.properties = properties; + } + + public void setServiceFacade(NiFiServiceFacade serviceFacade) { + this.serviceFacade = serviceFacade; + } + + public void setAuditService(AuditService auditService) { + this.auditService = auditService; + } + + public void setControllerServiceLookup(ControllerServiceLookup controllerServiceLookup) { + this.controllerServiceLookup = controllerServiceLookup; + } + + public void setReportingTaskProvider(ReportingTaskProvider reportingTaskProvider) { + this.reportingTaskProvider = reportingTaskProvider; + } + +} diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiWebContext.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiWebContext.java index d9fa9e3016..eaf457ed45 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiWebContext.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiWebContext.java @@ -35,7 +35,7 @@ import javax.ws.rs.core.Response; import org.apache.nifi.action.Action; import org.apache.nifi.action.Component; import org.apache.nifi.action.Operation; -import org.apache.nifi.action.component.details.ProcessorDetails; +import org.apache.nifi.action.component.details.ExtensionDetails; import org.apache.nifi.action.details.ConfigureDetails; import org.apache.nifi.admin.service.AuditService; import org.apache.nifi.cluster.manager.NodeResponse; @@ -49,7 +49,6 @@ import org.apache.nifi.web.api.dto.ProcessorConfigDTO; import org.apache.nifi.web.api.dto.ProcessorDTO; import org.apache.nifi.web.api.dto.RevisionDTO; import org.apache.nifi.web.api.entity.ProcessorEntity; -import org.apache.nifi.web.controller.ControllerFacade; import org.apache.nifi.web.util.WebUtils; import org.apache.commons.lang3.StringUtils; @@ -60,12 +59,14 @@ import org.springframework.security.core.Authentication; import org.springframework.security.core.context.SecurityContextHolder; import com.sun.jersey.core.util.MultivaluedMapImpl; +import org.apache.nifi.controller.ControllerServiceLookup; import org.apache.nifi.web.util.ClientResponseUtils; /** * Implements the NiFiWebContext interface to support a context in both * standalone and clustered environments. */ +@Deprecated public class StandardNiFiWebContext implements NiFiWebContext { private static final Logger logger = LoggerFactory.getLogger(StandardNiFiWebContext.class); @@ -76,16 +77,12 @@ public class StandardNiFiWebContext implements NiFiWebContext { private NiFiProperties properties; private NiFiServiceFacade serviceFacade; private WebClusterManager clusterManager; - private ControllerFacade controllerFacade; + private ControllerServiceLookup controllerServiceLookup; private AuditService auditService; @Override public ControllerService getControllerService(String serviceIdentifier) { - if (properties.isClusterManager()) { - return clusterManager.getControllerService(serviceIdentifier); - } else { - return controllerFacade.getControllerService(serviceIdentifier); - } + return controllerServiceLookup.getControllerService(serviceIdentifier); } @Override @@ -98,7 +95,7 @@ public class StandardNiFiWebContext implements NiFiWebContext { final Date now = new Date(); final Collection actions = new HashSet<>(processorActions.size()); for (final ProcessorConfigurationAction processorAction : processorActions) { - final ProcessorDetails processorDetails = new ProcessorDetails(); + final ExtensionDetails processorDetails = new ExtensionDetails(); processorDetails.setType(processorAction.getProcessorType()); final ConfigureDetails configureDetails = new ConfigureDetails(); @@ -199,7 +196,10 @@ public class StandardNiFiWebContext implements NiFiWebContext { } // return processor - final ProcessorEntity entity = nodeResponse.getClientResponse().getEntity(ProcessorEntity.class); + ProcessorEntity entity = (ProcessorEntity) nodeResponse.getUpdatedEntity(); + if (entity == null) { + entity = nodeResponse.getClientResponse().getEntity(ProcessorEntity.class); + } processor = entity.getProcessor(); } else { processor = serviceFacade.getProcessor(processorId); @@ -325,12 +325,12 @@ public class StandardNiFiWebContext implements NiFiWebContext { this.serviceFacade = serviceFacade; } - public void setControllerFacade(ControllerFacade controllerFacade) { - this.controllerFacade = controllerFacade; - } - public void setAuditService(AuditService auditService) { this.auditService = auditService; } + public void setControllerServiceLookup(ControllerServiceLookup controllerServiceLookup) { + this.controllerServiceLookup = controllerServiceLookup; + } + } diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ApplicationResource.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ApplicationResource.java index 1b9ae7def2..787fffaf45 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ApplicationResource.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ApplicationResource.java @@ -178,53 +178,55 @@ public abstract class ApplicationResource { // get cluster context from threadlocal ClusterContext clusterCtx = ClusterContextThreadLocal.getContext(); + if (clusterCtx != null) { + + // serialize cluster context + String serializedClusterContext = WebUtils.serializeObjectToHex(clusterCtx); + if (serializedClusterContext.length() > CLUSTER_CONTEXT_HEADER_VALUE_MAX_BYTES) { + /* + * Actions is the only field that can vary in size. If we have no + * actions and we exceeded the header size, then basic assumptions + * about the cluster context have been violated. + */ + if (clusterCtx.getActions().isEmpty()) { + throw new IllegalStateException( + String.format("Serialized Cluster context size '%d' is too big for response header", serializedClusterContext.length())); + } - // serialize cluster context - String serializedClusterContext = WebUtils.serializeObjectToHex(clusterCtx); - if (serializedClusterContext.length() > CLUSTER_CONTEXT_HEADER_VALUE_MAX_BYTES) { - /* - * Actions is the only field that can vary in size. If we have no - * actions and we exceeded the header size, then basic assumptions - * about the cluster context have been violated. - */ - if (clusterCtx.getActions().isEmpty()) { - throw new IllegalStateException( - String.format("Serialized Cluster context size '%d' is too big for response header", serializedClusterContext.length())); + // use the first action as the prototype for creating the "batch" action + Action prototypeAction = clusterCtx.getActions().get(0); + + // log the batched actions + StringBuilder loggedActions = new StringBuilder(); + createBatchedActionLogStatement(loggedActions, clusterCtx.getActions()); + logger.info(loggedActions.toString()); + + // remove current actions and replace with batch action + clusterCtx.getActions().clear(); + + // create the batch action + Action batchAction = new Action(); + batchAction.setOperation(Operation.Batch); + + // copy values from prototype action + batchAction.setTimestamp(prototypeAction.getTimestamp()); + batchAction.setUserDn(prototypeAction.getUserDn()); + batchAction.setUserName(prototypeAction.getUserName()); + batchAction.setSourceId(prototypeAction.getSourceId()); + batchAction.setSourceName(prototypeAction.getSourceName()); + batchAction.setSourceType(prototypeAction.getSourceType()); + + // add batch action + clusterCtx.getActions().add(batchAction); + + // create the final serialized copy of the cluster context + serializedClusterContext = WebUtils.serializeObjectToHex(clusterCtx); } - // use the first action as the prototype for creating the "batch" action - Action prototypeAction = clusterCtx.getActions().get(0); - - // log the batched actions - StringBuilder loggedActions = new StringBuilder(); - createBatchedActionLogStatement(loggedActions, clusterCtx.getActions()); - logger.info(loggedActions.toString()); - - // remove current actions and replace with batch action - clusterCtx.getActions().clear(); - - // create the batch action - Action batchAction = new Action(); - batchAction.setOperation(Operation.Batch); - - // copy values from prototype action - batchAction.setTimestamp(prototypeAction.getTimestamp()); - batchAction.setUserDn(prototypeAction.getUserDn()); - batchAction.setUserName(prototypeAction.getUserName()); - batchAction.setSourceId(prototypeAction.getSourceId()); - batchAction.setSourceName(prototypeAction.getSourceName()); - batchAction.setSourceType(prototypeAction.getSourceType()); - - // add batch action - clusterCtx.getActions().add(batchAction); - - // create the final serialized copy of the cluster context - serializedClusterContext = WebUtils.serializeObjectToHex(clusterCtx); + // put serialized cluster context in response header + response.header(WebClusterManager.CLUSTER_CONTEXT_HTTP_HEADER, serializedClusterContext); } - // put serialized cluster context in response header - response.header(WebClusterManager.CLUSTER_CONTEXT_HTTP_HEADER, serializedClusterContext); - return response; } diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ClusterResource.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ClusterResource.java index e87f388d7d..3a747820ec 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ClusterResource.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ClusterResource.java @@ -379,7 +379,7 @@ public class ClusterResource extends ApplicationResource { // update the revision RevisionDTO updatedRevision = new RevisionDTO(); updatedRevision.setClientId(revision.getClientId()); - updatedRevision.setVersion(controllerResponse.getRevision()); + updatedRevision.setVersion(controllerResponse.getVersion()); // generate the response entity final ProcessorEntity entity = new ProcessorEntity(); diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ConnectionResource.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ConnectionResource.java index a941444769..5d233f78c7 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ConnectionResource.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ConnectionResource.java @@ -450,7 +450,7 @@ public class ConnectionResource extends ApplicationResource { // get the updated revision final RevisionDTO updatedRevision = new RevisionDTO(); updatedRevision.setClientId(revision.getClientId()); - updatedRevision.setVersion(controllerResponse.getRevision()); + updatedRevision.setVersion(controllerResponse.getVersion()); // create the response entity ConnectionEntity entity = new ConnectionEntity(); @@ -684,7 +684,7 @@ public class ConnectionResource extends ApplicationResource { // get the updated revision final RevisionDTO updatedRevision = new RevisionDTO(); updatedRevision.setClientId(revision.getClientId()); - updatedRevision.setVersion(controllerResponse.getRevision()); + updatedRevision.setVersion(controllerResponse.getVersion()); // create the response entity ConnectionEntity entity = new ConnectionEntity(); @@ -742,7 +742,7 @@ public class ConnectionResource extends ApplicationResource { // create the revision final RevisionDTO updatedRevision = new RevisionDTO(); updatedRevision.setClientId(clientId.getClientId()); - updatedRevision.setVersion(controllerResponse.getRevision()); + updatedRevision.setVersion(controllerResponse.getVersion()); // create the response entity final ConnectionEntity entity = new ConnectionEntity(); diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ControllerResource.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ControllerResource.java index 3afe0e12cc..98f17d5444 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ControllerResource.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ControllerResource.java @@ -72,6 +72,8 @@ import org.apache.nifi.web.api.request.ClientIdParameter; import org.apache.nifi.web.api.request.IntegerParameter; import org.apache.nifi.web.api.request.LongParameter; import org.apache.commons.lang3.StringUtils; +import org.apache.nifi.web.api.entity.ControllerServiceTypesEntity; +import org.apache.nifi.web.api.entity.ReportingTaskTypesEntity; import org.codehaus.enunciate.jaxrs.TypeHint; import org.springframework.security.access.prepost.PreAuthorize; @@ -149,7 +151,7 @@ public class ControllerResource extends ApplicationResource { } /** - * Locates the Template sub-resource. + * Locates the Snippets sub-resource. * * @return */ @@ -157,6 +159,26 @@ public class ControllerResource extends ApplicationResource { public SnippetResource getSnippetResource() { return resourceContext.getResource(SnippetResource.class); } + + /** + * Locates the Controller Services sub-resource. + * + * @return + */ + @Path("/controller-services") + public ControllerServiceResource getControllerServiceResource() { + return resourceContext.getResource(ControllerServiceResource.class); + } + + /** + * Locates the Reporting Tasks sub-resource. + * + * @return + */ + @Path("/reporting-tasks") + public ReportingTaskResource getReportingTaskResource() { + return resourceContext.getResource(ReportingTaskResource.class); + } /** * Locates the Group sub-resource. @@ -303,7 +325,7 @@ public class ControllerResource extends ApplicationResource { // create the revision final RevisionDTO updatedRevision = new RevisionDTO(); updatedRevision.setClientId(clientId.getClientId()); - updatedRevision.setVersion(controllerResponse.getRevision()); + updatedRevision.setVersion(controllerResponse.getVersion()); // create the response entity final ProcessGroupEntity controllerEntity = new ProcessGroupEntity(); @@ -325,11 +347,6 @@ public class ControllerResource extends ApplicationResource { @PreAuthorize("hasAnyRole('ROLE_MONITOR', 'ROLE_DFM', 'ROLE_ADMIN')") @TypeHint(Entity.class) public Response getRevision() { - // replicate if cluster manager - if (properties.isClusterManager()) { - return clusterManager.applyRequest(HttpMethod.GET, getAbsolutePath(), getRequestParameters(true), getHeaders()).getResponse(); - } - // create the current revision final RevisionDTO revision = serviceFacade.getRevision(); @@ -595,7 +612,7 @@ public class ControllerResource extends ApplicationResource { // get the updated revision final RevisionDTO updatedRevision = new RevisionDTO(); updatedRevision.setClientId(revision.getClientId()); - updatedRevision.setVersion(controllerResponse.getRevision()); + updatedRevision.setVersion(controllerResponse.getVersion()); // create the response entity final ControllerConfigurationEntity entity = new ControllerConfigurationEntity(); @@ -713,6 +730,72 @@ public class ControllerResource extends ApplicationResource { // generate the response return clusterContext(generateOkResponse(entity)).build(); } + + /** + * Retrieves the types of controller services that this NiFi supports. + * + * @param clientId Optional client id. If the client id is not specified, a + * new one will be generated. This value (whether specified or generated) is + * included in the response. + * @return A controllerServicesTypesEntity. + */ + @GET + @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML}) + @Path("/controller-service-types") + @PreAuthorize("hasAnyRole('ROLE_MONITOR', 'ROLE_DFM', 'ROLE_ADMIN')") + @TypeHint(ControllerServiceTypesEntity.class) + public Response getControllerServiceTypes(@QueryParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) ClientIdParameter clientId) { + + // replicate if cluster manager + if (properties.isClusterManager()) { + return clusterManager.applyRequest(HttpMethod.GET, getAbsolutePath(), getRequestParameters(true), getHeaders()).getResponse(); + } + + // create the revision + final RevisionDTO revision = new RevisionDTO(); + revision.setClientId(clientId.getClientId()); + + // create response entity + final ControllerServiceTypesEntity entity = new ControllerServiceTypesEntity(); + entity.setRevision(revision); + entity.setControllerServiceTypes(serviceFacade.getControllerServiceTypes()); + + // generate the response + return clusterContext(generateOkResponse(entity)).build(); + } + + /** + * Retrieves the types of reporting tasks that this NiFi supports. + * + * @param clientId Optional client id. If the client id is not specified, a + * new one will be generated. This value (whether specified or generated) is + * included in the response. + * @return A controllerServicesTypesEntity. + */ + @GET + @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML}) + @Path("/reporting-task-types") + @PreAuthorize("hasAnyRole('ROLE_MONITOR', 'ROLE_DFM', 'ROLE_ADMIN')") + @TypeHint(ReportingTaskTypesEntity.class) + public Response getReportingTaskTypes(@QueryParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) ClientIdParameter clientId) { + + // replicate if cluster manager + if (properties.isClusterManager()) { + return clusterManager.applyRequest(HttpMethod.GET, getAbsolutePath(), getRequestParameters(true), getHeaders()).getResponse(); + } + + // create the revision + final RevisionDTO revision = new RevisionDTO(); + revision.setClientId(clientId.getClientId()); + + // create response entity + final ReportingTaskTypesEntity entity = new ReportingTaskTypesEntity(); + entity.setRevision(revision); + entity.setReportingTaskTypes(serviceFacade.getReportingTaskTypes()); + + // generate the response + return clusterContext(generateOkResponse(entity)).build(); + } /** * Retrieves the types of prioritizers that this NiFi supports. diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ControllerServiceResource.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ControllerServiceResource.java new file mode 100644 index 0000000000..1711f3cd67 --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ControllerServiceResource.java @@ -0,0 +1,803 @@ +/* + * 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.web.api; + +import java.net.URI; +import java.net.URISyntaxException; +import java.util.HashMap; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.UUID; +import javax.servlet.ServletContext; +import javax.servlet.http.HttpServletRequest; +import javax.ws.rs.Consumes; +import javax.ws.rs.DELETE; +import javax.ws.rs.DefaultValue; +import javax.ws.rs.FormParam; +import javax.ws.rs.GET; +import javax.ws.rs.HttpMethod; +import javax.ws.rs.POST; +import javax.ws.rs.PUT; +import javax.ws.rs.Path; +import javax.ws.rs.PathParam; +import javax.ws.rs.Produces; +import javax.ws.rs.QueryParam; +import javax.ws.rs.WebApplicationException; +import javax.ws.rs.core.Context; +import javax.ws.rs.core.MediaType; +import javax.ws.rs.core.MultivaluedMap; +import javax.ws.rs.core.Response; +import org.apache.nifi.cluster.manager.impl.WebClusterManager; +import org.apache.nifi.util.NiFiProperties; +import org.apache.nifi.web.ConfigurationSnapshot; +import org.apache.nifi.web.NiFiServiceFacade; +import org.apache.nifi.web.Revision; +import org.apache.nifi.web.api.dto.RevisionDTO; +import org.apache.nifi.web.api.request.ClientIdParameter; +import org.apache.nifi.web.api.request.LongParameter; +import org.apache.commons.lang3.StringUtils; +import org.apache.nifi.controller.ScheduledState; +import org.apache.nifi.controller.service.ControllerServiceState; +import org.apache.nifi.ui.extension.UiExtension; +import org.apache.nifi.ui.extension.UiExtensionMapping; +import org.apache.nifi.web.UiExtensionType; +import static org.apache.nifi.web.api.ApplicationResource.CLIENT_ID; +import org.apache.nifi.web.api.dto.ControllerServiceDTO; +import org.apache.nifi.web.api.dto.ControllerServiceReferencingComponentDTO; +import org.apache.nifi.web.api.dto.PropertyDescriptorDTO; +import org.apache.nifi.web.api.entity.ControllerServiceEntity; +import org.apache.nifi.web.api.entity.ControllerServiceReferencingComponentsEntity; +import org.apache.nifi.web.api.entity.ControllerServicesEntity; +import org.apache.nifi.web.api.entity.PropertyDescriptorEntity; +import org.apache.nifi.web.util.Availability; +import org.codehaus.enunciate.jaxrs.TypeHint; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.springframework.security.access.prepost.PreAuthorize; + +/** + * RESTful endpoint for managing a Controller Service. + */ +public class ControllerServiceResource extends ApplicationResource { + + private static final Logger logger = LoggerFactory.getLogger(ControllerServiceResource.class); + + private NiFiServiceFacade serviceFacade; + private WebClusterManager clusterManager; + private NiFiProperties properties; + + @Context + private ServletContext servletContext; + + /** + * Populates the uri for the specified controller service. + * + * @param controllerServices + * @return + */ + private Set populateRemainingControllerServicesContent(final String availability, final Set controllerServices) { + for (ControllerServiceDTO controllerService : controllerServices) { + populateRemainingControllerServiceContent(availability, controllerService); + } + return controllerServices; + } + + /** + * Populates the uri for the specified controller service. + */ + private ControllerServiceDTO populateRemainingControllerServiceContent(final String availability, final ControllerServiceDTO controllerService) { + // populate the controller service href + controllerService.setUri(generateResourceUri("controller", "controller-services", availability, controllerService.getId())); + controllerService.setAvailability(availability); + + // see if this processor has any ui extensions + final UiExtensionMapping uiExtensionMapping = (UiExtensionMapping) servletContext.getAttribute("nifi-ui-extensions"); + if (uiExtensionMapping.hasUiExtension(controllerService.getType())) { + final List uiExtensions = uiExtensionMapping.getUiExtension(controllerService.getType()); + for (final UiExtension uiExtension : uiExtensions) { + if (UiExtensionType.ControllerServiceConfiguration.equals(uiExtension.getExtensionType())) { + controllerService.setCustomUiUrl(uiExtension.getContextPath() + "/configure"); + } + } + } + + return controllerService; + } + + /** + * Parses the availability and ensure that the specified availability makes sense for the + * given NiFi instance. + * + * @param availability + * @return + */ + private Availability parseAvailability(final String availability) { + final Availability avail; + try { + avail = Availability.valueOf(availability.toUpperCase()); + } catch (IllegalArgumentException iae) { + throw new IllegalArgumentException(String.format("Availability: Value must be one of [%s]", StringUtils.join(Availability.values(), ", "))); + } + + // ensure this nifi is an NCM is specifying NCM availability + if (!properties.isClusterManager() && Availability.NCM.equals(avail)) { + throw new IllegalArgumentException("Availability of NCM is only applicable when the NiFi instance is the cluster manager."); + } + + return avail; + } + + /** + * Retrieves all the of controller services in this NiFi. + * + * @param clientId Optional client id. If the client id is not specified, a + * new one will be generated. This value (whether specified or generated) is + * included in the response. + * @param availability Whether the controller service is available on the NCM only (ncm) or on the + * nodes only (node). If this instance is not clustered all services should use the node availability. + * @return A controllerServicesEntity. + */ + @GET + @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML}) + @Path("/{availability}") + @PreAuthorize("hasAnyRole('ROLE_MONITOR', 'ROLE_DFM', 'ROLE_ADMIN')") + @TypeHint(ControllerServicesEntity.class) + public Response getControllerServices(@QueryParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) ClientIdParameter clientId, @PathParam("availability") String availability) { + final Availability avail = parseAvailability(availability); + + // replicate if cluster manager + if (properties.isClusterManager() && Availability.NODE.equals(avail)) { + return clusterManager.applyRequest(HttpMethod.GET, getAbsolutePath(), getRequestParameters(true), getHeaders()).getResponse(); + } + + // get all the controller services + final Set controllerServices = populateRemainingControllerServicesContent(availability, serviceFacade.getControllerServices()); + + // create the revision + final RevisionDTO revision = new RevisionDTO(); + revision.setClientId(clientId.getClientId()); + + // create the response entity + final ControllerServicesEntity entity = new ControllerServicesEntity(); + entity.setRevision(revision); + entity.setControllerServices(controllerServices); + + // generate the response + return clusterContext(generateOkResponse(entity)).build(); + } + + /** + * Creates a new controller service. + * + * @param httpServletRequest + * @param version The revision is used to verify the client is working with + * the latest version of the flow. + * @param clientId Optional client id. If the client id is not specified, a + * new one will be generated. This value (whether specified or generated) is + * included in the response. + * @param availability Whether the controller service is available on the NCM only (ncm) or on the + * nodes only (node). If this instance is not clustered all services should use the node availability. + * @param type The type of controller service to create. + * @return A controllerServiceEntity. + */ + @POST + @Consumes(MediaType.APPLICATION_FORM_URLENCODED) + @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML}) + @Path("/{availability}") + @PreAuthorize("hasRole('ROLE_DFM')") + @TypeHint(ControllerServiceEntity.class) + public Response createControllerService( + @Context HttpServletRequest httpServletRequest, + @FormParam(VERSION) LongParameter version, + @FormParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) ClientIdParameter clientId, + @PathParam("availability") String availability, + @FormParam("type") String type) { + + // create the controller service DTO + final ControllerServiceDTO controllerServiceDTO = new ControllerServiceDTO(); + controllerServiceDTO.setType(type); + + // create the revision + final RevisionDTO revision = new RevisionDTO(); + revision.setClientId(clientId.getClientId()); + if (version != null) { + revision.setVersion(version.getLong()); + } + + // create the controller service entity + final ControllerServiceEntity controllerServiceEntity = new ControllerServiceEntity(); + controllerServiceEntity.setRevision(revision); + controllerServiceEntity.setControllerService(controllerServiceDTO); + + return createControllerService(httpServletRequest, availability, controllerServiceEntity); + } + + /** + * Creates a new Controller Service. + * + * @param httpServletRequest + * @param availability Whether the controller service is available on the NCM only (ncm) or on the + * nodes only (node). If this instance is not clustered all services should use the node availability. + * @param controllerServiceEntity A controllerServiceEntity. + * @return A controllerServiceEntity. + */ + @POST + @Consumes({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML}) + @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML}) + @Path("/{availability}") + @PreAuthorize("hasRole('ROLE_DFM')") + @TypeHint(ControllerServiceEntity.class) + public Response createControllerService( + @Context HttpServletRequest httpServletRequest, + @PathParam("availability") String availability, + ControllerServiceEntity controllerServiceEntity) { + + final Availability avail = parseAvailability(availability); + + if (controllerServiceEntity == null || controllerServiceEntity.getControllerService()== null) { + throw new IllegalArgumentException("Controller service details must be specified."); + } + + if (controllerServiceEntity.getRevision() == null) { + throw new IllegalArgumentException("Revision must be specified."); + } + + if (controllerServiceEntity.getControllerService().getId() != null) { + throw new IllegalArgumentException("Controller service ID cannot be specified."); + } + + if (StringUtils.isBlank(controllerServiceEntity.getControllerService().getType())) { + throw new IllegalArgumentException("The type of controller service to create must be specified."); + } + + // get the revision + final RevisionDTO revision = controllerServiceEntity.getRevision(); + + // if cluster manager, convert POST to PUT (to maintain same ID across nodes) and replicate + if (properties.isClusterManager() && Availability.NODE.equals(avail)) { + // create ID for resource + final String id = UUID.randomUUID().toString(); + + // set ID for resource + controllerServiceEntity.getControllerService().setId(id); + + // convert POST request to PUT request to force entity ID to be the same across nodes + URI putUri = null; + try { + putUri = new URI(getAbsolutePath().toString() + "/" + id); + } catch (final URISyntaxException e) { + throw new WebApplicationException(e); + } + + // change content type to JSON for serializing entity + final Map headersToOverride = new HashMap<>(); + headersToOverride.put("content-type", MediaType.APPLICATION_JSON); + + // replicate put request + return (Response) clusterManager.applyRequest(HttpMethod.PUT, putUri, updateClientId(controllerServiceEntity), getHeaders(headersToOverride)).getResponse(); + } + + // handle expects request (usually from the cluster manager) + final String expects = httpServletRequest.getHeader(WebClusterManager.NCM_EXPECTS_HTTP_HEADER); + if (expects != null) { + return generateContinueResponse().build(); + } + + // create the controller service and generate the json + final ConfigurationSnapshot controllerResponse = serviceFacade.createControllerService( + new Revision(revision.getVersion(), revision.getClientId()), controllerServiceEntity.getControllerService()); + final ControllerServiceDTO controllerService = controllerResponse.getConfiguration(); + + // get the updated revision + final RevisionDTO updatedRevision = new RevisionDTO(); + updatedRevision.setClientId(revision.getClientId()); + updatedRevision.setVersion(controllerResponse.getVersion()); + + // build the response entity + final ControllerServiceEntity entity = new ControllerServiceEntity(); + entity.setRevision(updatedRevision); + entity.setControllerService(populateRemainingControllerServiceContent(availability, controllerService)); + + // build the response + return clusterContext(generateCreatedResponse(URI.create(controllerService.getUri()), entity)).build(); + } + + /** + * Retrieves the specified controller service. + * + * @param clientId Optional client id. If the client id is not specified, a + * new one will be generated. This value (whether specified or generated) is + * included in the response. + * @param availability Whether the controller service is available on the NCM only (ncm) or on the + * nodes only (node). If this instance is not clustered all services should use the node availability. + * @param id The id of the controller service to retrieve + * @return A controllerServiceEntity. + */ + @GET + @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML}) + @Path("/{availability}/{id}") + @PreAuthorize("hasAnyRole('ROLE_MONITOR', 'ROLE_DFM', 'ROLE_ADMIN')") + @TypeHint(ControllerServiceEntity.class) + public Response getControllerService(@QueryParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) ClientIdParameter clientId, + @PathParam("availability") String availability, @PathParam("id") String id) { + + final Availability avail = parseAvailability(availability); + + // replicate if cluster manager + if (properties.isClusterManager() && Availability.NODE.equals(avail)) { + return clusterManager.applyRequest(HttpMethod.GET, getAbsolutePath(), getRequestParameters(true), getHeaders()).getResponse(); + } + + // get the controller service + final ControllerServiceDTO controllerService = serviceFacade.getControllerService(id); + + // create the revision + final RevisionDTO revision = new RevisionDTO(); + revision.setClientId(clientId.getClientId()); + + // create the response entity + final ControllerServiceEntity entity = new ControllerServiceEntity(); + entity.setRevision(revision); + entity.setControllerService(populateRemainingControllerServiceContent(availability, controllerService)); + + return clusterContext(generateOkResponse(entity)).build(); + } + + /** + * Returns the descriptor for the specified property. + * + * @param clientId Optional client id. If the client id is not specified, a + * new one will be generated. This value (whether specified or generated) is + * included in the response. + * @param availability + * @param id The id of the controller service. + * @param propertyName The property + * @return a propertyDescriptorEntity + */ + @GET + @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML}) + @Path("/{availability}/{id}/descriptors") + @PreAuthorize("hasAnyRole('ROLE_MONITOR', 'ROLE_DFM', 'ROLE_ADMIN')") + @TypeHint(PropertyDescriptorEntity.class) + public Response getPropertyDescriptor( + @QueryParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) ClientIdParameter clientId, + @PathParam("availability") String availability, @PathParam("id") String id, + @QueryParam("propertyName") String propertyName) { + + final Availability avail = parseAvailability(availability); + + // ensure the property name is specified + if (propertyName == null) { + throw new IllegalArgumentException("The property name must be specified."); + } + + // replicate if cluster manager and service is on node + if (properties.isClusterManager() && Availability.NODE.equals(avail)) { + return clusterManager.applyRequest(HttpMethod.GET, getAbsolutePath(), getRequestParameters(true), getHeaders()).getResponse(); + } + + // get the property descriptor + final PropertyDescriptorDTO descriptor = serviceFacade.getControllerServicePropertyDescriptor(id, propertyName); + + // create the revision + final RevisionDTO revision = new RevisionDTO(); + revision.setClientId(clientId.getClientId()); + + // generate the response entity + final PropertyDescriptorEntity entity = new PropertyDescriptorEntity(); + entity.setRevision(revision); + entity.setPropertyDescriptor(descriptor); + + // generate the response + return clusterContext(generateOkResponse(entity)).build(); + } + + /** + * Retrieves the references of the specified controller service. + * + * @param clientId Optional client id. If the client id is not specified, a + * new one will be generated. This value (whether specified or generated) is + * included in the response. + * @param availability Whether the controller service is available on the NCM only (ncm) or on the + * nodes only (node). If this instance is not clustered all services should use the node availability. + * @param id The id of the controller service to retrieve + * @return A controllerServiceEntity. + */ + @GET + @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML}) + @Path("/{availability}/{id}/references") + @PreAuthorize("hasAnyRole('ROLE_MONITOR', 'ROLE_DFM', 'ROLE_ADMIN')") + @TypeHint(ControllerServiceEntity.class) + public Response getControllerServiceReferences( + @QueryParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) ClientIdParameter clientId, + @PathParam("availability") String availability, @PathParam("id") String id) { + + final Availability avail = parseAvailability(availability); + + // replicate if cluster manager + if (properties.isClusterManager() && Availability.NODE.equals(avail)) { + return clusterManager.applyRequest(HttpMethod.GET, getAbsolutePath(), getRequestParameters(true), getHeaders()).getResponse(); + } + + // get the controller service + final Set controllerServiceReferences = serviceFacade.getControllerServiceReferencingComponents(id); + + // create the revision + final RevisionDTO revision = new RevisionDTO(); + revision.setClientId(clientId.getClientId()); + + // create the response entity + final ControllerServiceReferencingComponentsEntity entity = new ControllerServiceReferencingComponentsEntity(); + entity.setRevision(revision); + entity.setControllerServiceReferencingComponents(controllerServiceReferences); + + return clusterContext(generateOkResponse(entity)).build(); + } + + /** + * Updates the references of the specified controller service. + * + * @param httpServletRequest + * @param version The revision is used to verify the client is working with + * the latest version of the flow. + * @param clientId Optional client id. If the client id is not specified, a + * new one will be generated. This value (whether specified or generated) is + * included in the response. + * @param availability Whether the controller service is available on the NCM only (ncm) or on the + * nodes only (node). If this instance is not clustered all services should use the node availability. + * @param id The id of the controller service to retrieve + * @param state Sets the state of referencing components. A value of RUNNING or STOPPED will update + * referencing schedulable components (Processors and Reporting Tasks). A value of ENABLED or + * DISABLED will update referencing controller services. + * @return A controllerServiceEntity. + */ + @PUT + @Consumes(MediaType.APPLICATION_FORM_URLENCODED) + @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML}) + @Path("/{availability}/{id}/references") + @PreAuthorize("hasAnyRole('ROLE_MONITOR', 'ROLE_DFM', 'ROLE_ADMIN')") + @TypeHint(ControllerServiceEntity.class) + public Response updateControllerServiceReferences( + @Context HttpServletRequest httpServletRequest, + @FormParam(VERSION) LongParameter version, + @FormParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) ClientIdParameter clientId, + @PathParam("availability") String availability, @PathParam("id") String id, + @FormParam("state") @DefaultValue(StringUtils.EMPTY) String state) { + + // parse the state to determine the desired action + + // need to consider controller service state first as it shares a state with + // scheduled state (disabled) which is applicable for referencing services + // but not referencing schedulable components + ControllerServiceState controllerServiceState = null; + try { + controllerServiceState = ControllerServiceState.valueOf(state); + } catch (final IllegalArgumentException iae) { + // ignore + } + + ScheduledState scheduledState = null; + try { + scheduledState = ScheduledState.valueOf(state); + } catch (final IllegalArgumentException iae) { + // ignore + } + + // ensure an action has been specified + if (scheduledState == null && controllerServiceState == null) { + throw new IllegalArgumentException("Must specify the updated state. To update referencing Processors " + + "and Reporting Tasks the state should be RUNNING or STOPPED. To update the referencing Controller Services the " + + "state should be ENABLED or DISABLED."); + } + + // ensure the controller service state is not ENABLING or DISABLING + if (controllerServiceState != null && (ControllerServiceState.ENABLING.equals(controllerServiceState) || ControllerServiceState.DISABLING.equals(controllerServiceState))) { + throw new IllegalArgumentException("Cannot set the referencing services to ENABLING or DISABLING"); + } + + // determine the availability + final Availability avail = parseAvailability(availability); + + // replicate if cluster manager + if (properties.isClusterManager() && Availability.NODE.equals(avail)) { + return clusterManager.applyRequest(HttpMethod.PUT, getAbsolutePath(), getRequestParameters(true), getHeaders()).getResponse(); + } + + // handle expects request (usually from the cluster manager) + final String expects = httpServletRequest.getHeader(WebClusterManager.NCM_EXPECTS_HTTP_HEADER); + if (expects != null) { + serviceFacade.verifyUpdateControllerServiceReferencingComponents(id, scheduledState, controllerServiceState); + return generateContinueResponse().build(); + } + + // determine the specified version + Long clientVersion = null; + if (version != null) { + clientVersion = version.getLong(); + } + + // get the controller service + final ConfigurationSnapshot> response = + serviceFacade.updateControllerServiceReferencingComponents(new Revision(clientVersion, clientId.getClientId()), id, scheduledState, controllerServiceState); + + // create the revision + final RevisionDTO revision = new RevisionDTO(); + revision.setClientId(clientId.getClientId()); + revision.setVersion(response.getVersion()); + + // create the response entity + final ControllerServiceReferencingComponentsEntity entity = new ControllerServiceReferencingComponentsEntity(); + entity.setRevision(revision); + entity.setControllerServiceReferencingComponents(response.getConfiguration()); + + return clusterContext(generateOkResponse(entity)).build(); + } + + /** + * Updates the specified controller service. + * + * @param httpServletRequest + * @param version The revision is used to verify the client is working with + * the latest version of the flow. + * @param clientId Optional client id. If the client id is not specified, a + * new one will be generated. This value (whether specified or generated) is + * included in the response. + * @param availability Whether the controller service is available on the NCM only (ncm) or on the + * nodes only (node). If this instance is not clustered all services should use the node availability. + * @param id The id of the controller service to update. + * @param name The name of the controller service + * @param annotationData The annotation data for the controller service + * @param comments The comments for the controller service + * @param state The state of this controller service. Should be ENABLED or DISABLED. + * @param markedForDeletion Array of property names whose value should be removed. + * @param formParams Additionally, the processor properties and styles are + * specified in the form parameters. Because the property names and styles + * differ from processor to processor they are specified in a map-like + * fashion: + *
      + *
        + *
      • properties[required.file.path]=/path/to/file
      • + *
      • properties[required.hostname]=localhost
      • + *
      • properties[required.port]=80
      • + *
      • properties[optional.file.path]=/path/to/file
      • + *
      • properties[optional.hostname]=localhost
      • + *
      • properties[optional.port]=80
      • + *
      • properties[user.defined.pattern]=^.*?s.*$
      • + *
      + * @return A controllerServiceEntity. + */ + @PUT + @Consumes(MediaType.APPLICATION_FORM_URLENCODED) + @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML}) + @Path("/{availability}/{id}") + @PreAuthorize("hasRole('ROLE_DFM')") + @TypeHint(ControllerServiceEntity.class) + public Response updateControllerService( + @Context HttpServletRequest httpServletRequest, + @FormParam(VERSION) LongParameter version, + @FormParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) ClientIdParameter clientId, + @PathParam("availability") String availability, @PathParam("id") String id, @FormParam("name") String name, + @FormParam("annotationData") String annotationData, @FormParam("comments") String comments, + @FormParam("state") String state, @FormParam("markedForDeletion[]") List markedForDeletion, + MultivaluedMap formParams) { + + // create collections for holding the controller service properties + final Map updatedProperties = new LinkedHashMap<>(); + + // go through each parameter and look for processor properties + for (String parameterName : formParams.keySet()) { + if (StringUtils.isNotBlank(parameterName)) { + // see if the parameter name starts with an expected parameter type... + // if so, store the parameter name and value in the corresponding collection + if (parameterName.startsWith("properties")) { + final int startIndex = StringUtils.indexOf(parameterName, "["); + final int endIndex = StringUtils.lastIndexOf(parameterName, "]"); + if (startIndex != -1 && endIndex != -1) { + final String propertyName = StringUtils.substring(parameterName, startIndex + 1, endIndex); + updatedProperties.put(propertyName, formParams.getFirst(parameterName)); + } + } + } + } + + // set the properties to remove + for (String propertyToDelete : markedForDeletion) { + updatedProperties.put(propertyToDelete, null); + } + + // create the controller service DTO + final ControllerServiceDTO controllerServiceDTO = new ControllerServiceDTO(); + controllerServiceDTO.setId(id); + controllerServiceDTO.setName(name); + controllerServiceDTO.setAnnotationData(annotationData); + controllerServiceDTO.setComments(comments); + controllerServiceDTO.setState(state); + + // only set the properties when appropriate + if (!updatedProperties.isEmpty()) { + controllerServiceDTO.setProperties(updatedProperties); + } + + // create the revision + final RevisionDTO revision = new RevisionDTO(); + revision.setClientId(clientId.getClientId()); + if (version != null) { + revision.setVersion(version.getLong()); + } + + // create the controller service entity + final ControllerServiceEntity controllerServiceEntity = new ControllerServiceEntity(); + controllerServiceEntity.setRevision(revision); + controllerServiceEntity.setControllerService(controllerServiceDTO); + + // update the controller service + return updateControllerService(httpServletRequest, availability, id, controllerServiceEntity); + } + + /** + * Updates the specified a new Controller Service. + * + * @param httpServletRequest + * @param availability Whether the controller service is available on the NCM only (ncm) or on the + * nodes only (node). If this instance is not clustered all services should use the node availability. + * @param id The id of the controller service to update. + * @param controllerServiceEntity A controllerServiceEntity. + * @return A controllerServiceEntity. + */ + @PUT + @Consumes({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML}) + @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML}) + @Path("/{availability}/{id}") + @PreAuthorize("hasRole('ROLE_DFM')") + @TypeHint(ControllerServiceEntity.class) + public Response updateControllerService( + @Context HttpServletRequest httpServletRequest, + @PathParam("availability") String availability, + @PathParam("id") String id, + ControllerServiceEntity controllerServiceEntity) { + + final Availability avail = parseAvailability(availability); + + if (controllerServiceEntity == null || controllerServiceEntity.getControllerService()== null) { + throw new IllegalArgumentException("Controller service details must be specified."); + } + + if (controllerServiceEntity.getRevision() == null) { + throw new IllegalArgumentException("Revision must be specified."); + } + + // ensure the ids are the same + final ControllerServiceDTO requestControllerServiceDTO = controllerServiceEntity.getControllerService(); + if (!id.equals(requestControllerServiceDTO.getId())) { + throw new IllegalArgumentException(String.format("The controller service id (%s) in the request body does not equal the " + + "controller service id of the requested resource (%s).", requestControllerServiceDTO.getId(), id)); + } + + // replicate if cluster manager + if (properties.isClusterManager() && Availability.NODE.equals(avail)) { + // change content type to JSON for serializing entity + final Map headersToOverride = new HashMap<>(); + headersToOverride.put("content-type", MediaType.APPLICATION_JSON); + + // replicate the request + return clusterManager.applyRequest(HttpMethod.PUT, getAbsolutePath(), updateClientId(controllerServiceEntity), getHeaders(headersToOverride)).getResponse(); + } + + // handle expects request (usually from the cluster manager) + final String expects = httpServletRequest.getHeader(WebClusterManager.NCM_EXPECTS_HTTP_HEADER); + if (expects != null) { + serviceFacade.verifyUpdateControllerService(requestControllerServiceDTO); + return generateContinueResponse().build(); + } + + // update the controller service + final RevisionDTO revision = controllerServiceEntity.getRevision(); + final ConfigurationSnapshot controllerResponse = serviceFacade.updateControllerService( + new Revision(revision.getVersion(), revision.getClientId()), requestControllerServiceDTO); + + // get the results + final ControllerServiceDTO responseControllerServiceDTO = controllerResponse.getConfiguration(); + + // get the updated revision + final RevisionDTO updatedRevision = new RevisionDTO(); + updatedRevision.setClientId(revision.getClientId()); + updatedRevision.setVersion(controllerResponse.getVersion()); + + // build the response entity + final ControllerServiceEntity entity = new ControllerServiceEntity(); + entity.setRevision(updatedRevision); + entity.setControllerService(populateRemainingControllerServiceContent(availability, responseControllerServiceDTO)); + + return clusterContext(generateOkResponse(entity)).build(); + } + + /** + * Removes the specified controller service. + * + * @param httpServletRequest + * @param version The revision is used to verify the client is working with + * the latest version of the flow. + * @param clientId Optional client id. If the client id is not specified, a + * new one will be generated. This value (whether specified or generated) is + * included in the response. + * @param availability Whether the controller service is available on the NCM only (ncm) or on the + * nodes only (node). If this instance is not clustered all services should use the node availability. + * @param id The id of the controller service to remove. + * @return A entity containing the client id and an updated revision. + */ + @DELETE + @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML}) + @Path("/{availability}/{id}") + @PreAuthorize("hasRole('ROLE_DFM')") + @TypeHint(ControllerServiceEntity.class) + public Response removeControllerService( + @Context HttpServletRequest httpServletRequest, + @QueryParam(VERSION) LongParameter version, + @QueryParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) ClientIdParameter clientId, + @PathParam("availability") String availability, @PathParam("id") String id) { + + final Availability avail = parseAvailability(availability); + + // replicate if cluster manager + if (properties.isClusterManager() && Availability.NODE.equals(avail)) { + return clusterManager.applyRequest(HttpMethod.DELETE, getAbsolutePath(), getRequestParameters(true), getHeaders()).getResponse(); + } + + // handle expects request (usually from the cluster manager) + final String expects = httpServletRequest.getHeader(WebClusterManager.NCM_EXPECTS_HTTP_HEADER); + if (expects != null) { + serviceFacade.verifyDeleteControllerService(id); + return generateContinueResponse().build(); + } + + // determine the specified version + Long clientVersion = null; + if (version != null) { + clientVersion = version.getLong(); + } + + // delete the specified controller service + final ConfigurationSnapshot controllerResponse = serviceFacade.deleteControllerService(new Revision(clientVersion, clientId.getClientId()), id); + + // get the updated revision + final RevisionDTO revision = new RevisionDTO(); + revision.setClientId(clientId.getClientId()); + revision.setVersion(controllerResponse.getVersion()); + + // build the response entity + final ControllerServiceEntity entity = new ControllerServiceEntity(); + entity.setRevision(revision); + + return clusterContext(generateOkResponse(entity)).build(); + } + + // setters + + public void setServiceFacade(NiFiServiceFacade serviceFacade) { + this.serviceFacade = serviceFacade; + } + + public void setClusterManager(WebClusterManager clusterManager) { + this.clusterManager = clusterManager; + } + + public void setProperties(NiFiProperties properties) { + this.properties = properties; + } +} diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/FunnelResource.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/FunnelResource.java index 4406c2efba..3492de2093 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/FunnelResource.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/FunnelResource.java @@ -244,7 +244,7 @@ public class FunnelResource extends ApplicationResource { // get the updated revision final RevisionDTO updatedRevision = new RevisionDTO(); updatedRevision.setClientId(revision.getClientId()); - updatedRevision.setVersion(controllerResponse.getRevision()); + updatedRevision.setVersion(controllerResponse.getVersion()); // build the response entity final FunnelEntity entity = new FunnelEntity(); @@ -408,7 +408,7 @@ public class FunnelResource extends ApplicationResource { // get the updated revision final RevisionDTO updatedRevision = new RevisionDTO(); updatedRevision.setClientId(revision.getClientId()); - updatedRevision.setVersion(controllerResponse.getRevision()); + updatedRevision.setVersion(controllerResponse.getVersion()); // build the response entity final FunnelEntity entity = new FunnelEntity(); @@ -465,7 +465,7 @@ public class FunnelResource extends ApplicationResource { // get the updated revision final RevisionDTO revision = new RevisionDTO(); revision.setClientId(clientId.getClientId()); - revision.setVersion(controllerResponse.getRevision()); + revision.setVersion(controllerResponse.getVersion()); // build the response entity final FunnelEntity entity = new FunnelEntity(); diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/HistoryResource.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/HistoryResource.java index 38806eb01c..0f60f52fad 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/HistoryResource.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/HistoryResource.java @@ -37,7 +37,7 @@ import org.apache.nifi.web.api.dto.RevisionDTO; import org.apache.nifi.web.api.dto.action.ActionDTO; import org.apache.nifi.web.api.dto.action.HistoryDTO; import org.apache.nifi.web.api.dto.action.HistoryQueryDTO; -import org.apache.nifi.web.api.entity.ProcessorHistoryEntity; +import org.apache.nifi.web.api.entity.ComponentHistoryEntity; import org.codehaus.enunciate.jaxrs.TypeHint; import org.springframework.security.access.prepost.PreAuthorize; @@ -245,7 +245,7 @@ public class HistoryResource extends ApplicationResource { @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML}) @PreAuthorize("hasAnyRole('ROLE_MONITOR', 'ROLE_DFM', 'ROLE_ADMIN')") @Path("/processors/{processorId}") - @TypeHint(ProcessorHistoryEntity.class) + @TypeHint(ComponentHistoryEntity.class) public Response getProcessorHistory( @QueryParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) ClientIdParameter clientId, @PathParam("processorId") final String processorId) { @@ -255,14 +255,76 @@ public class HistoryResource extends ApplicationResource { revision.setClientId(clientId.getClientId()); // create the response entity - final ProcessorHistoryEntity entity = new ProcessorHistoryEntity(); + final ComponentHistoryEntity entity = new ComponentHistoryEntity(); entity.setRevision(revision); - entity.setProcessorHistory(serviceFacade.getProcessorHistory(processorId)); + entity.setComponentHistory(serviceFacade.getComponentHistory(processorId)); // generate the response return generateOkResponse(entity).build(); } + + /** + * Gets the actions for the specified controller service. + * + * @param clientId Optional client id. If the client id is not specified, a + * new one will be generated. This value (whether specified or generated) is + * included in the response. + * @param controllerServiceId The id of the controller service. + * @return An componentHistoryEntity. + */ + @GET + @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML}) + @PreAuthorize("hasAnyRole('ROLE_MONITOR', 'ROLE_DFM', 'ROLE_ADMIN')") + @Path("/controller-services/{controllerServiceId}") + @TypeHint(ComponentHistoryEntity.class) + public Response getControllerServiceHistory( + @QueryParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) ClientIdParameter clientId, + @PathParam("controllerServiceId") final String controllerServiceId) { + // create the revision + final RevisionDTO revision = new RevisionDTO(); + revision.setClientId(clientId.getClientId()); + + // create the response entity + final ComponentHistoryEntity entity = new ComponentHistoryEntity(); + entity.setRevision(revision); + entity.setComponentHistory(serviceFacade.getComponentHistory(controllerServiceId)); + + // generate the response + return generateOkResponse(entity).build(); + } + + /** + * Gets the actions for the specified reporting task. + * + * @param clientId Optional client id. If the client id is not specified, a + * new one will be generated. This value (whether specified or generated) is + * included in the response. + * @param reportingTaskId The id of the reporting task. + * @return An componentHistoryEntity. + */ + @GET + @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML}) + @PreAuthorize("hasAnyRole('ROLE_MONITOR', 'ROLE_DFM', 'ROLE_ADMIN')") + @Path("/reporting-tasks/{reportingTaskId}") + @TypeHint(ComponentHistoryEntity.class) + public Response getReportingTaskHistory( + @QueryParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) ClientIdParameter clientId, + @PathParam("reportingTaskId") final String reportingTaskId) { + + // create the revision + final RevisionDTO revision = new RevisionDTO(); + revision.setClientId(clientId.getClientId()); + + // create the response entity + final ComponentHistoryEntity entity = new ComponentHistoryEntity(); + entity.setRevision(revision); + entity.setComponentHistory(serviceFacade.getComponentHistory(reportingTaskId)); + + // generate the response + return generateOkResponse(entity).build(); + } + /* setters */ public void setServiceFacade(NiFiServiceFacade serviceFacade) { this.serviceFacade = serviceFacade; diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/InputPortResource.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/InputPortResource.java index 58c3c9e89a..f3a6326604 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/InputPortResource.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/InputPortResource.java @@ -251,7 +251,7 @@ public class InputPortResource extends ApplicationResource { // get the updated revision final RevisionDTO updatedRevision = new RevisionDTO(); updatedRevision.setClientId(revision.getClientId()); - updatedRevision.setVersion(controllerResponse.getRevision()); + updatedRevision.setVersion(controllerResponse.getVersion()); // build the response entity final InputPortEntity entity = new InputPortEntity(); @@ -446,7 +446,7 @@ public class InputPortResource extends ApplicationResource { // get the updated revision final RevisionDTO updatedRevision = new RevisionDTO(); updatedRevision.setClientId(revision.getClientId()); - updatedRevision.setVersion(controllerResponse.getRevision()); + updatedRevision.setVersion(controllerResponse.getVersion()); // build the response entity final InputPortEntity entity = new InputPortEntity(); @@ -503,7 +503,7 @@ public class InputPortResource extends ApplicationResource { // get the updated revision final RevisionDTO revision = new RevisionDTO(); revision.setClientId(clientId.getClientId()); - revision.setVersion(controllerResponse.getRevision()); + revision.setVersion(controllerResponse.getVersion()); // build the response entity final InputPortEntity entity = new InputPortEntity(); diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/LabelResource.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/LabelResource.java index 9a61cfcc6e..6435671d49 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/LabelResource.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/LabelResource.java @@ -260,7 +260,7 @@ public class LabelResource extends ApplicationResource { // get the updated revision final RevisionDTO updatedRevision = new RevisionDTO(); updatedRevision.setClientId(revision.getClientId()); - updatedRevision.setVersion(controllerResponse.getRevision()); + updatedRevision.setVersion(controllerResponse.getVersion()); // build the response entity final LabelEntity entity = new LabelEntity(); @@ -463,7 +463,7 @@ public class LabelResource extends ApplicationResource { // get the updated revision final RevisionDTO updatedRevision = new RevisionDTO(); updatedRevision.setClientId(revision.getClientId()); - updatedRevision.setVersion(controllerResponse.getRevision()); + updatedRevision.setVersion(controllerResponse.getVersion()); // build the response entity final LabelEntity entity = new LabelEntity(); @@ -519,7 +519,7 @@ public class LabelResource extends ApplicationResource { // get the updated revision final RevisionDTO revision = new RevisionDTO(); revision.setClientId(clientId.getClientId()); - revision.setVersion(controllerResponse.getRevision()); + revision.setVersion(controllerResponse.getVersion()); // build the response entity final LabelEntity entity = new LabelEntity(); diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/OutputPortResource.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/OutputPortResource.java index 224ab18588..a9dce5fdfa 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/OutputPortResource.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/OutputPortResource.java @@ -251,7 +251,7 @@ public class OutputPortResource extends ApplicationResource { // get the updated revision final RevisionDTO updatedRevision = new RevisionDTO(); updatedRevision.setClientId(revision.getClientId()); - updatedRevision.setVersion(controllerResponse.getRevision()); + updatedRevision.setVersion(controllerResponse.getVersion()); // build the response entity final OutputPortEntity entity = new OutputPortEntity(); @@ -447,7 +447,7 @@ public class OutputPortResource extends ApplicationResource { // get the updated revision final RevisionDTO updatedRevision = new RevisionDTO(); updatedRevision.setClientId(revision.getClientId()); - updatedRevision.setVersion(controllerResponse.getRevision()); + updatedRevision.setVersion(controllerResponse.getVersion()); // build the response entity final OutputPortEntity entity = new OutputPortEntity(); @@ -504,7 +504,7 @@ public class OutputPortResource extends ApplicationResource { // get the updated revision final RevisionDTO revision = new RevisionDTO(); revision.setClientId(clientId.getClientId()); - revision.setVersion(controllerResponse.getRevision()); + revision.setVersion(controllerResponse.getVersion()); // build the response entity final OutputPortEntity entity = new OutputPortEntity(); diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ProcessGroupResource.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ProcessGroupResource.java index 6439bda1a6..1bf3f7779a 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ProcessGroupResource.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ProcessGroupResource.java @@ -287,7 +287,7 @@ public class ProcessGroupResource extends ApplicationResource { // get the updated revision final RevisionDTO revision = new RevisionDTO(); revision.setClientId(clientId.getClientId()); - revision.setVersion(controllerResponse.getRevision()); + revision.setVersion(controllerResponse.getVersion()); // create the response entity final ProcessGroupEntity processGroupEntity = new ProcessGroupEntity(); @@ -365,7 +365,7 @@ public class ProcessGroupResource extends ApplicationResource { // get the updated revision final RevisionDTO revision = new RevisionDTO(); revision.setClientId(clientId.getClientId()); - revision.setVersion(controllerResponse.getRevision()); + revision.setVersion(controllerResponse.getVersion()); // create the response entity final FlowSnippetEntity entity = new FlowSnippetEntity(); @@ -441,7 +441,7 @@ public class ProcessGroupResource extends ApplicationResource { // get the updated revision final RevisionDTO revision = new RevisionDTO(); revision.setClientId(clientId.getClientId()); - revision.setVersion(response.getRevision()); + revision.setVersion(response.getVersion()); // create the response entity final FlowSnippetEntity entity = new FlowSnippetEntity(); @@ -559,7 +559,7 @@ public class ProcessGroupResource extends ApplicationResource { // get the updated revision final RevisionDTO updatedRevision = new RevisionDTO(); updatedRevision.setClientId(revision.getClientId()); - updatedRevision.setVersion(response.getRevision()); + updatedRevision.setVersion(response.getVersion()); // create the response entity final ProcessGroupEntity entity = new ProcessGroupEntity(); @@ -616,7 +616,7 @@ public class ProcessGroupResource extends ApplicationResource { // create the revision final RevisionDTO revision = new RevisionDTO(); revision.setClientId(clientId.getClientId()); - revision.setVersion(controllerResponse.getRevision()); + revision.setVersion(controllerResponse.getVersion()); // create the response entity final ProcessGroupEntity processGroupEntity = new ProcessGroupEntity(); @@ -795,7 +795,7 @@ public class ProcessGroupResource extends ApplicationResource { // get the updated revision final RevisionDTO updatedRevision = new RevisionDTO(); updatedRevision.setClientId(revision.getClientId()); - updatedRevision.setVersion(controllerResponse.getRevision()); + updatedRevision.setVersion(controllerResponse.getVersion()); // create the response entity final ProcessGroupEntity entity = new ProcessGroupEntity(); @@ -931,7 +931,7 @@ public class ProcessGroupResource extends ApplicationResource { // create the revision final RevisionDTO updatedRevision = new RevisionDTO(); updatedRevision.setClientId(revision.getClientId()); - updatedRevision.setVersion(response.getRevision()); + updatedRevision.setVersion(response.getVersion()); // create the response entity final ProcessGroupEntity entity = new ProcessGroupEntity(); @@ -989,7 +989,7 @@ public class ProcessGroupResource extends ApplicationResource { // get the updated revision final RevisionDTO revision = new RevisionDTO(); revision.setClientId(clientId.getClientId()); - revision.setVersion(controllerResponse.getRevision()); + revision.setVersion(controllerResponse.getVersion()); // create the response entity final ProcessGroupEntity entity = new ProcessGroupEntity(); diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ProcessorResource.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ProcessorResource.java index b11c40a0e9..31ab10b6d9 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ProcessorResource.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ProcessorResource.java @@ -70,6 +70,11 @@ import org.apache.nifi.web.api.request.IntegerParameter; import org.apache.nifi.web.api.request.LongParameter; import org.apache.commons.lang3.StringUtils; +import org.apache.nifi.ui.extension.UiExtension; +import org.apache.nifi.ui.extension.UiExtensionMapping; +import org.apache.nifi.web.UiExtensionType; +import org.apache.nifi.web.api.dto.PropertyDescriptorDTO; +import org.apache.nifi.web.api.entity.PropertyDescriptorEntity; import org.codehaus.enunciate.jaxrs.TypeHint; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -118,9 +123,21 @@ public class ProcessorResource extends ApplicationResource { // get the config details and see if there is a custom ui for this processor type ProcessorConfigDTO config = processor.getConfig(); if (config != null) { + // consider legacy custom ui fist String customUiUrl = servletContext.getInitParameter(processor.getType()); if (StringUtils.isNotBlank(customUiUrl)) { config.setCustomUiUrl(customUiUrl); + } else { + // see if this processor has any ui extensions + final UiExtensionMapping uiExtensionMapping = (UiExtensionMapping) servletContext.getAttribute("nifi-ui-extensions"); + if (uiExtensionMapping.hasUiExtension(processor.getType())) { + final List uiExtensions = uiExtensionMapping.getUiExtension(processor.getType()); + for (final UiExtension uiExtension : uiExtensions) { + if (UiExtensionType.ProcessorConfiguration.equals(uiExtension.getExtensionType())) { + config.setCustomUiUrl(uiExtension.getContextPath() + "/configure"); + } + } + } } } @@ -245,6 +262,10 @@ public class ProcessorResource extends ApplicationResource { if (processorEntity.getProcessor().getId() != null) { throw new IllegalArgumentException("Processor ID cannot be specified."); } + + if (StringUtils.isBlank(processorEntity.getProcessor().getType())) { + throw new IllegalArgumentException("The type of processor to create must be specified."); + } // if cluster manager, convert POST to PUT (to maintain same ID across nodes) and replicate if (properties.isClusterManager()) { @@ -288,7 +309,7 @@ public class ProcessorResource extends ApplicationResource { // get the updated revision final RevisionDTO updatedRevision = new RevisionDTO(); updatedRevision.setClientId(revision.getClientId()); - updatedRevision.setVersion(controllerResponse.getRevision()); + updatedRevision.setVersion(controllerResponse.getVersion()); // generate the response entity final ProcessorEntity entity = new ProcessorEntity(); @@ -373,6 +394,51 @@ public class ProcessorResource extends ApplicationResource { // generate the response return clusterContext(generateOkResponse(entity)).build(); } + + /** + * Returns the descriptor for the specified property. + * + * @param clientId Optional client id. If the client id is not specified, a + * new one will be generated. This value (whether specified or generated) is + * included in the response. + * @param id The id of the processor + * @param propertyName The property + * @return a propertyDescriptorEntity + */ + @GET + @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML}) + @Path("/{id}/descriptors") + @PreAuthorize("hasAnyRole('ROLE_MONITOR', 'ROLE_DFM', 'ROLE_ADMIN')") + @TypeHint(PropertyDescriptorEntity.class) + public Response getPropertyDescriptor( + @QueryParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) ClientIdParameter clientId, + @PathParam("id") String id, @QueryParam("propertyName") String propertyName) { + + // ensure the property name is specified + if (propertyName == null) { + throw new IllegalArgumentException("The property name must be specified."); + } + + // replicate if cluster manager + if (properties.isClusterManager()) { + return clusterManager.applyRequest(HttpMethod.GET, getAbsolutePath(), getRequestParameters(true), getHeaders()).getResponse(); + } + + // get the property descriptor + final PropertyDescriptorDTO descriptor = serviceFacade.getProcessorPropertyDescriptor(groupId, id, propertyName); + + // create the revision + final RevisionDTO revision = new RevisionDTO(); + revision.setClientId(clientId.getClientId()); + + // generate the response entity + final PropertyDescriptorEntity entity = new PropertyDescriptorEntity(); + entity.setRevision(revision); + entity.setPropertyDescriptor(descriptor); + + // generate the response + return clusterContext(generateOkResponse(entity)).build(); + } /** * Updates the specified processor with the specified values. @@ -607,7 +673,7 @@ public class ProcessorResource extends ApplicationResource { // get the updated revision final RevisionDTO updatedRevision = new RevisionDTO(); updatedRevision.setClientId(revision.getClientId()); - updatedRevision.setVersion(controllerResponse.getRevision()); + updatedRevision.setVersion(controllerResponse.getVersion()); // generate the response entity final ProcessorEntity entity = new ProcessorEntity(); @@ -664,7 +730,7 @@ public class ProcessorResource extends ApplicationResource { // get the updated revision final RevisionDTO updatedRevision = new RevisionDTO(); updatedRevision.setClientId(clientId.getClientId()); - updatedRevision.setVersion(controllerResponse.getRevision()); + updatedRevision.setVersion(controllerResponse.getVersion()); // generate the response entity final ProcessorEntity entity = new ProcessorEntity(); diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/RemoteProcessGroupResource.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/RemoteProcessGroupResource.java index 27fa29236f..4e15c36743 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/RemoteProcessGroupResource.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/RemoteProcessGroupResource.java @@ -396,7 +396,7 @@ public class RemoteProcessGroupResource extends ApplicationResource { // get the updated revision final RevisionDTO updatedRevision = new RevisionDTO(); updatedRevision.setClientId(revision.getClientId()); - updatedRevision.setVersion(controllerResponse.getRevision()); + updatedRevision.setVersion(controllerResponse.getVersion()); // build the response entity final RemoteProcessGroupEntity entity = new RemoteProcessGroupEntity(); @@ -452,7 +452,7 @@ public class RemoteProcessGroupResource extends ApplicationResource { // get the updated revision final RevisionDTO revision = new RevisionDTO(); revision.setClientId(clientId.getClientId()); - revision.setVersion(controllerResponse.getRevision()); + revision.setVersion(controllerResponse.getVersion()); // create the response entity final RemoteProcessGroupEntity entity = new RemoteProcessGroupEntity(); @@ -586,7 +586,7 @@ public class RemoteProcessGroupResource extends ApplicationResource { // get the updated revision final RevisionDTO updatedRevision = new RevisionDTO(); updatedRevision.setClientId(revision.getClientId()); - updatedRevision.setVersion(controllerResponse.getRevision()); + updatedRevision.setVersion(controllerResponse.getVersion()); // build the response entity final RemoteProcessGroupPortEntity entity = new RemoteProcessGroupPortEntity(); @@ -720,7 +720,7 @@ public class RemoteProcessGroupResource extends ApplicationResource { // get the updated revision final RevisionDTO updatedRevision = new RevisionDTO(); updatedRevision.setClientId(revision.getClientId()); - updatedRevision.setVersion(controllerResponse.getRevision()); + updatedRevision.setVersion(controllerResponse.getVersion()); // build the response entity RemoteProcessGroupPortEntity entity = new RemoteProcessGroupPortEntity(); @@ -890,7 +890,7 @@ public class RemoteProcessGroupResource extends ApplicationResource { // get the updated revision final RevisionDTO updatedRevision = new RevisionDTO(); updatedRevision.setClientId(revision.getClientId()); - updatedRevision.setVersion(controllerResponse.getRevision()); + updatedRevision.setVersion(controllerResponse.getVersion()); // build the response entity final RemoteProcessGroupEntity entity = new RemoteProcessGroupEntity(); diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ReportingTaskResource.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ReportingTaskResource.java new file mode 100644 index 0000000000..38ddc36628 --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ReportingTaskResource.java @@ -0,0 +1,663 @@ +/* + * 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.web.api; + +import java.net.URI; +import java.net.URISyntaxException; +import java.util.HashMap; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.UUID; +import javax.servlet.ServletContext; +import javax.servlet.http.HttpServletRequest; +import javax.ws.rs.Consumes; +import javax.ws.rs.DELETE; +import javax.ws.rs.DefaultValue; +import javax.ws.rs.FormParam; +import javax.ws.rs.GET; +import javax.ws.rs.HttpMethod; +import javax.ws.rs.POST; +import javax.ws.rs.PUT; +import javax.ws.rs.Path; +import javax.ws.rs.PathParam; +import javax.ws.rs.Produces; +import javax.ws.rs.QueryParam; +import javax.ws.rs.WebApplicationException; +import javax.ws.rs.core.Context; +import javax.ws.rs.core.MediaType; +import javax.ws.rs.core.MultivaluedMap; +import javax.ws.rs.core.Response; +import org.apache.nifi.cluster.manager.impl.WebClusterManager; +import org.apache.nifi.util.NiFiProperties; +import org.apache.nifi.web.ConfigurationSnapshot; +import org.apache.nifi.web.NiFiServiceFacade; +import org.apache.nifi.web.Revision; +import org.apache.nifi.web.api.dto.RevisionDTO; +import org.apache.nifi.web.api.request.ClientIdParameter; +import org.apache.nifi.web.api.request.LongParameter; +import org.apache.commons.lang3.StringUtils; +import org.apache.nifi.ui.extension.UiExtension; +import org.apache.nifi.ui.extension.UiExtensionMapping; +import org.apache.nifi.web.UiExtensionType; +import static org.apache.nifi.web.api.ApplicationResource.CLIENT_ID; +import org.apache.nifi.web.api.dto.PropertyDescriptorDTO; +import org.apache.nifi.web.api.dto.ReportingTaskDTO; +import org.apache.nifi.web.api.entity.PropertyDescriptorEntity; +import org.apache.nifi.web.api.entity.ReportingTaskEntity; +import org.apache.nifi.web.api.entity.ReportingTasksEntity; +import org.apache.nifi.web.util.Availability; +import org.codehaus.enunciate.jaxrs.TypeHint; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.springframework.security.access.prepost.PreAuthorize; + +/** + * RESTful endpoint for managing a Reporting Task. + */ +public class ReportingTaskResource extends ApplicationResource { + + private static final Logger logger = LoggerFactory.getLogger(ReportingTaskResource.class); + + private NiFiServiceFacade serviceFacade; + private WebClusterManager clusterManager; + private NiFiProperties properties; + + @Context + private ServletContext servletContext; + + /** + * Populates the uri for the specified reporting task. + * + * @param reportingTasks + * @return + */ + private Set populateRemainingReportingTasksContent(final String availability, final Set reportingTasks) { + for (ReportingTaskDTO reportingTask : reportingTasks) { + populateRemainingReportingTaskContent(availability, reportingTask); + } + return reportingTasks; + } + + /** + * Populates the uri for the specified reporting task. + */ + private ReportingTaskDTO populateRemainingReportingTaskContent(final String availability, final ReportingTaskDTO reportingTask) { + // populate the reporting task href + reportingTask.setUri(generateResourceUri("controller", "reporting-tasks", availability, reportingTask.getId())); + reportingTask.setAvailability(availability); + + // see if this processor has any ui extensions + final UiExtensionMapping uiExtensionMapping = (UiExtensionMapping) servletContext.getAttribute("nifi-ui-extensions"); + if (uiExtensionMapping.hasUiExtension(reportingTask.getType())) { + final List uiExtensions = uiExtensionMapping.getUiExtension(reportingTask.getType()); + for (final UiExtension uiExtension : uiExtensions) { + if (UiExtensionType.ReportingTaskConfiguration.equals(uiExtension.getExtensionType())) { + reportingTask.setCustomUiUrl(uiExtension.getContextPath() + "/configure"); + } + } + } + + return reportingTask; + } + + /** + * Parses the availability and ensure that the specified availability makes sense for the + * given NiFi instance. + * + * @param availability + * @return + */ + private Availability parseAvailability(final String availability) { + final Availability avail; + try { + avail = Availability.valueOf(availability.toUpperCase()); + } catch (IllegalArgumentException iae) { + throw new IllegalArgumentException(String.format("Availability: Value must be one of [%s]", StringUtils.join(Availability.values(), ", "))); + } + + // ensure this nifi is an NCM is specifying NCM availability + if (!properties.isClusterManager() && Availability.NCM.equals(avail)) { + throw new IllegalArgumentException("Availability of NCM is only applicable when the NiFi instance is the cluster manager."); + } + + return avail; + } + + /** + * Retrieves all the of reporting tasks in this NiFi. + * + * @param clientId Optional client id. If the client id is not specified, a + * new one will be generated. This value (whether specified or generated) is + * included in the response. + * @param availability Whether the reporting task is available on the NCM only (ncm) or on the + * nodes only (node). If this instance is not clustered all tasks should use the node availability. + * @return A reportingTasksEntity. + */ + @GET + @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML}) + @Path("/{availability}") + @PreAuthorize("hasAnyRole('ROLE_MONITOR', 'ROLE_DFM', 'ROLE_ADMIN')") + @TypeHint(ReportingTasksEntity.class) + public Response getReportingTasks(@QueryParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) ClientIdParameter clientId, @PathParam("availability") String availability) { + final Availability avail = parseAvailability(availability); + + // replicate if cluster manager + if (properties.isClusterManager() && Availability.NODE.equals(avail)) { + return clusterManager.applyRequest(HttpMethod.GET, getAbsolutePath(), getRequestParameters(true), getHeaders()).getResponse(); + } + + // get all the reporting tasks + final Set reportingTasks = populateRemainingReportingTasksContent(availability, serviceFacade.getReportingTasks()); + + // create the revision + final RevisionDTO revision = new RevisionDTO(); + revision.setClientId(clientId.getClientId()); + + // create the response entity + final ReportingTasksEntity entity = new ReportingTasksEntity(); + entity.setRevision(revision); + entity.setReportingTasks(reportingTasks); + + // generate the response + return clusterContext(generateOkResponse(entity)).build(); + } + + /** + * Creates a new reporting task. + * + * @param httpServletRequest + * @param version The revision is used to verify the client is working with + * the latest version of the flow. + * @param clientId Optional client id. If the client id is not specified, a + * new one will be generated. This value (whether specified or generated) is + * included in the response. + * @param availability Whether the reporting task is available on the NCM only (ncm) or on the + * nodes only (node). If this instance is not clustered all tasks should use the node availability. + * @param type The type of reporting task to create. + * @return A reportingTaskEntity. + */ + @POST + @Consumes(MediaType.APPLICATION_FORM_URLENCODED) + @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML}) + @Path("/{availability}") + @PreAuthorize("hasRole('ROLE_DFM')") + @TypeHint(ReportingTaskEntity.class) + public Response createReportingTask( + @Context HttpServletRequest httpServletRequest, + @FormParam(VERSION) LongParameter version, + @FormParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) ClientIdParameter clientId, + @PathParam("availability") String availability, + @FormParam("type") String type) { + + // create the reporting task DTO + final ReportingTaskDTO reportingTaskDTO = new ReportingTaskDTO(); + reportingTaskDTO.setType(type); + + // create the revision + final RevisionDTO revision = new RevisionDTO(); + revision.setClientId(clientId.getClientId()); + if (version != null) { + revision.setVersion(version.getLong()); + } + + // create the reporting task entity + final ReportingTaskEntity reportingTaskEntity = new ReportingTaskEntity(); + reportingTaskEntity.setRevision(revision); + reportingTaskEntity.setReportingTask(reportingTaskDTO); + + return createReportingTask(httpServletRequest, availability, reportingTaskEntity); + } + + /** + * Creates a new Reporting Task. + * + * @param httpServletRequest + * @param availability Whether the reporting task is available on the NCM only (ncm) or on the + * nodes only (node). If this instance is not clustered all tasks should use the node availability. + * @param reportingTaskEntity A reportingTaskEntity. + * @return A reportingTaskEntity. + */ + @POST + @Consumes({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML}) + @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML}) + @Path("/{availability}") + @PreAuthorize("hasRole('ROLE_DFM')") + @TypeHint(ReportingTaskEntity.class) + public Response createReportingTask( + @Context HttpServletRequest httpServletRequest, + @PathParam("availability") String availability, + ReportingTaskEntity reportingTaskEntity) { + + final Availability avail = parseAvailability(availability); + + if (reportingTaskEntity == null || reportingTaskEntity.getReportingTask()== null) { + throw new IllegalArgumentException("Reporting task details must be specified."); + } + + if (reportingTaskEntity.getRevision() == null) { + throw new IllegalArgumentException("Revision must be specified."); + } + + if (reportingTaskEntity.getReportingTask().getId() != null) { + throw new IllegalArgumentException("Reporting task ID cannot be specified."); + } + + if (StringUtils.isBlank(reportingTaskEntity.getReportingTask().getType())) { + throw new IllegalArgumentException("The type of reporting task to create must be specified."); + } + + // get the revision + final RevisionDTO revision = reportingTaskEntity.getRevision(); + + // if cluster manager, convert POST to PUT (to maintain same ID across nodes) and replicate + if (properties.isClusterManager() && Availability.NODE.equals(avail)) { + // create ID for resource + final String id = UUID.randomUUID().toString(); + + // set ID for resource + reportingTaskEntity.getReportingTask().setId(id); + + // convert POST request to PUT request to force entity ID to be the same across nodes + URI putUri = null; + try { + putUri = new URI(getAbsolutePath().toString() + "/" + id); + } catch (final URISyntaxException e) { + throw new WebApplicationException(e); + } + + // change content type to JSON for serializing entity + final Map headersToOverride = new HashMap<>(); + headersToOverride.put("content-type", MediaType.APPLICATION_JSON); + + // replicate put request + return (Response) clusterManager.applyRequest(HttpMethod.PUT, putUri, updateClientId(reportingTaskEntity), getHeaders(headersToOverride)).getResponse(); + } + + // handle expects request (usually from the cluster manager) + final String expects = httpServletRequest.getHeader(WebClusterManager.NCM_EXPECTS_HTTP_HEADER); + if (expects != null) { + return generateContinueResponse().build(); + } + + // create the reporting task and generate the json + final ConfigurationSnapshot controllerResponse = serviceFacade.createReportingTask( + new Revision(revision.getVersion(), revision.getClientId()), reportingTaskEntity.getReportingTask()); + final ReportingTaskDTO reportingTask = controllerResponse.getConfiguration(); + + // get the updated revision + final RevisionDTO updatedRevision = new RevisionDTO(); + updatedRevision.setClientId(revision.getClientId()); + updatedRevision.setVersion(controllerResponse.getVersion()); + + // build the response entity + final ReportingTaskEntity entity = new ReportingTaskEntity(); + entity.setRevision(updatedRevision); + entity.setReportingTask(populateRemainingReportingTaskContent(availability, reportingTask)); + + // build the response + return clusterContext(generateCreatedResponse(URI.create(reportingTask.getUri()), entity)).build(); + } + + /** + * Retrieves the specified reporting task. + * + * @param clientId Optional client id. If the client id is not specified, a + * new one will be generated. This value (whether specified or generated) is + * included in the response. + * @param availability Whether the reporting task is available on the NCM only (ncm) or on the + * nodes only (node). If this instance is not clustered all tasks should use the node availability. + * @param id The id of the reporting task to retrieve + * @return A reportingTaskEntity. + */ + @GET + @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML}) + @Path("/{availability}/{id}") + @PreAuthorize("hasAnyRole('ROLE_MONITOR', 'ROLE_DFM', 'ROLE_ADMIN')") + @TypeHint(ReportingTaskEntity.class) + public Response getReportingTask(@QueryParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) ClientIdParameter clientId, + @PathParam("availability") String availability, @PathParam("id") String id) { + + final Availability avail = parseAvailability(availability); + + // replicate if cluster manager + if (properties.isClusterManager() && Availability.NODE.equals(avail)) { + return clusterManager.applyRequest(HttpMethod.GET, getAbsolutePath(), getRequestParameters(true), getHeaders()).getResponse(); + } + + // get the reporting task + final ReportingTaskDTO reportingTask = serviceFacade.getReportingTask(id); + + // create the revision + final RevisionDTO revision = new RevisionDTO(); + revision.setClientId(clientId.getClientId()); + + // create the response entity + final ReportingTaskEntity entity = new ReportingTaskEntity(); + entity.setRevision(revision); + entity.setReportingTask(populateRemainingReportingTaskContent(availability, reportingTask)); + + return clusterContext(generateOkResponse(entity)).build(); + } + + /** + * Returns the descriptor for the specified property. + * + * @param clientId Optional client id. If the client id is not specified, a + * new one will be generated. This value (whether specified or generated) is + * included in the response. + * @param availability + * @param id The id of the reporting task. + * @param propertyName The property + * @return a propertyDescriptorEntity + */ + @GET + @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML}) + @Path("/{availability}/{id}/descriptors") + @PreAuthorize("hasAnyRole('ROLE_MONITOR', 'ROLE_DFM', 'ROLE_ADMIN')") + @TypeHint(PropertyDescriptorEntity.class) + public Response getPropertyDescriptor( + @QueryParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) ClientIdParameter clientId, + @PathParam("availability") String availability, @PathParam("id") String id, + @QueryParam("propertyName") String propertyName) { + + final Availability avail = parseAvailability(availability); + + // ensure the property name is specified + if (propertyName == null) { + throw new IllegalArgumentException("The property name must be specified."); + } + + // replicate if cluster manager and task is on node + if (properties.isClusterManager() && Availability.NODE.equals(avail)) { + return clusterManager.applyRequest(HttpMethod.GET, getAbsolutePath(), getRequestParameters(true), getHeaders()).getResponse(); + } + + // get the property descriptor + final PropertyDescriptorDTO descriptor = serviceFacade.getReportingTaskPropertyDescriptor(id, propertyName); + + // create the revision + final RevisionDTO revision = new RevisionDTO(); + revision.setClientId(clientId.getClientId()); + + // generate the response entity + final PropertyDescriptorEntity entity = new PropertyDescriptorEntity(); + entity.setRevision(revision); + entity.setPropertyDescriptor(descriptor); + + // generate the response + return clusterContext(generateOkResponse(entity)).build(); + } + + /** + * Updates the specified reporting task. + * + * @param httpServletRequest + * @param version The revision is used to verify the client is working with + * the latest version of the flow. + * @param clientId Optional client id. If the client id is not specified, a + * new one will be generated. This value (whether specified or generated) is + * included in the response. + * @param availability Whether the reporting task is available on the NCM only (ncm) or on the + * nodes only (node). If this instance is not clustered all tasks should use the node availability. + * @param id The id of the reporting task to update. + * @param name The name of the reporting task + * @param annotationData The annotation data for the reporting task + * @param markedForDeletion Array of property names whose value should be removed. + * @param state The updated scheduled state + * @param schedulingStrategy The scheduling strategy for this reporting task + * @param schedulingPeriod The scheduling period for this reporting task + * @param comments The comments for this reporting task + * @param formParams Additionally, the processor properties and styles are + * specified in the form parameters. Because the property names and styles + * differ from processor to processor they are specified in a map-like + * fashion: + *
      + *
        + *
      • properties[required.file.path]=/path/to/file
      • + *
      • properties[required.hostname]=localhost
      • + *
      • properties[required.port]=80
      • + *
      • properties[optional.file.path]=/path/to/file
      • + *
      • properties[optional.hostname]=localhost
      • + *
      • properties[optional.port]=80
      • + *
      • properties[user.defined.pattern]=^.*?s.*$
      • + *
      + * @return A reportingTaskEntity. + */ + @PUT + @Consumes(MediaType.APPLICATION_FORM_URLENCODED) + @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML}) + @Path("/{availability}/{id}") + @PreAuthorize("hasRole('ROLE_DFM')") + @TypeHint(ReportingTaskEntity.class) + public Response updateReportingTask( + @Context HttpServletRequest httpServletRequest, + @FormParam(VERSION) LongParameter version, + @FormParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) ClientIdParameter clientId, + @PathParam("availability") String availability, @PathParam("id") String id, @FormParam("name") String name, + @FormParam("annotationData") String annotationData, @FormParam("markedForDeletion[]") List markedForDeletion, + @FormParam("state") String state, @FormParam("schedulingStrategy") String schedulingStrategy, + @FormParam("schedulingPeriod") String schedulingPeriod, @FormParam("comments") String comments, + MultivaluedMap formParams) { + + // create collections for holding the reporting task properties + final Map updatedProperties = new LinkedHashMap<>(); + + // go through each parameter and look for processor properties + for (String parameterName : formParams.keySet()) { + if (StringUtils.isNotBlank(parameterName)) { + // see if the parameter name starts with an expected parameter type... + // if so, store the parameter name and value in the corresponding collection + if (parameterName.startsWith("properties")) { + final int startIndex = StringUtils.indexOf(parameterName, "["); + final int endIndex = StringUtils.lastIndexOf(parameterName, "]"); + if (startIndex != -1 && endIndex != -1) { + final String propertyName = StringUtils.substring(parameterName, startIndex + 1, endIndex); + updatedProperties.put(propertyName, formParams.getFirst(parameterName)); + } + } + } + } + + // set the properties to remove + for (String propertyToDelete : markedForDeletion) { + updatedProperties.put(propertyToDelete, null); + } + + // create the reporting task DTO + final ReportingTaskDTO reportingTaskDTO = new ReportingTaskDTO(); + reportingTaskDTO.setId(id); + reportingTaskDTO.setName(name); + reportingTaskDTO.setState(state); + reportingTaskDTO.setSchedulingStrategy(schedulingStrategy); + reportingTaskDTO.setSchedulingPeriod(schedulingPeriod); + reportingTaskDTO.setAnnotationData(annotationData); + reportingTaskDTO.setComments(comments); + + // only set the properties when appropriate + if (!updatedProperties.isEmpty()) { + reportingTaskDTO.setProperties(updatedProperties); + } + + // create the revision + final RevisionDTO revision = new RevisionDTO(); + revision.setClientId(clientId.getClientId()); + if (version != null) { + revision.setVersion(version.getLong()); + } + + // create the reporting task entity + final ReportingTaskEntity reportingTaskEntity = new ReportingTaskEntity(); + reportingTaskEntity.setRevision(revision); + reportingTaskEntity.setReportingTask(reportingTaskDTO); + + // update the reporting task + return updateReportingTask(httpServletRequest, availability, id, reportingTaskEntity); + } + + /** + * Updates the specified a Reporting Task. + * + * @param httpServletRequest + * @param availability Whether the reporting task is available on the NCM only (ncm) or on the + * nodes only (node). If this instance is not clustered all tasks should use the node availability. + * @param id The id of the reporting task to update. + * @param reportingTaskEntity A reportingTaskEntity. + * @return A reportingTaskEntity. + */ + @PUT + @Consumes({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML}) + @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML}) + @Path("/{availability}/{id}") + @PreAuthorize("hasRole('ROLE_DFM')") + @TypeHint(ReportingTaskEntity.class) + public Response updateReportingTask( + @Context HttpServletRequest httpServletRequest, + @PathParam("availability") String availability, + @PathParam("id") String id, + ReportingTaskEntity reportingTaskEntity) { + + final Availability avail = parseAvailability(availability); + + if (reportingTaskEntity == null || reportingTaskEntity.getReportingTask() == null) { + throw new IllegalArgumentException("Reporting task details must be specified."); + } + + if (reportingTaskEntity.getRevision() == null) { + throw new IllegalArgumentException("Revision must be specified."); + } + + // ensure the ids are the same + final ReportingTaskDTO requestReportingTaskDTO = reportingTaskEntity.getReportingTask(); + if (!id.equals(requestReportingTaskDTO.getId())) { + throw new IllegalArgumentException(String.format("The reporting task id (%s) in the request body does not equal the " + + "reporting task id of the requested resource (%s).", requestReportingTaskDTO.getId(), id)); + } + + // replicate if cluster manager + if (properties.isClusterManager() && Availability.NODE.equals(avail)) { + // change content type to JSON for serializing entity + final Map headersToOverride = new HashMap<>(); + headersToOverride.put("content-type", MediaType.APPLICATION_JSON); + + // replicate the request + return clusterManager.applyRequest(HttpMethod.PUT, getAbsolutePath(), updateClientId(reportingTaskEntity), getHeaders(headersToOverride)).getResponse(); + } + + // handle expects request (usually from the cluster manager) + final String expects = httpServletRequest.getHeader(WebClusterManager.NCM_EXPECTS_HTTP_HEADER); + if (expects != null) { + serviceFacade.verifyUpdateReportingTask(requestReportingTaskDTO); + return generateContinueResponse().build(); + } + + // update the reporting task + final RevisionDTO revision = reportingTaskEntity.getRevision(); + final ConfigurationSnapshot controllerResponse = serviceFacade.updateReportingTask( + new Revision(revision.getVersion(), revision.getClientId()), requestReportingTaskDTO); + + // get the results + final ReportingTaskDTO responseReportingTaskDTO = controllerResponse.getConfiguration(); + + // get the updated revision + final RevisionDTO updatedRevision = new RevisionDTO(); + updatedRevision.setClientId(revision.getClientId()); + updatedRevision.setVersion(controllerResponse.getVersion()); + + // build the response entity + final ReportingTaskEntity entity = new ReportingTaskEntity(); + entity.setRevision(updatedRevision); + entity.setReportingTask(populateRemainingReportingTaskContent(availability, responseReportingTaskDTO)); + + return clusterContext(generateOkResponse(entity)).build(); + } + + /** + * Removes the specified reporting task. + * + * @param httpServletRequest + * @param version The revision is used to verify the client is working with + * the latest version of the flow. + * @param clientId Optional client id. If the client id is not specified, a + * new one will be generated. This value (whether specified or generated) is + * included in the response. + * @param availability Whether the reporting task is available on the NCM only (ncm) or on the + * nodes only (node). If this instance is not clustered all tasks should use the node availability. + * @param id The id of the reporting task to remove. + * @return A entity containing the client id and an updated revision. + */ + @DELETE + @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML}) + @Path("/{availability}/{id}") + @PreAuthorize("hasRole('ROLE_DFM')") + @TypeHint(ReportingTaskEntity.class) + public Response removeReportingTask( + @Context HttpServletRequest httpServletRequest, + @QueryParam(VERSION) LongParameter version, + @QueryParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) ClientIdParameter clientId, + @PathParam("availability") String availability, @PathParam("id") String id) { + + final Availability avail = parseAvailability(availability); + + // replicate if cluster manager + if (properties.isClusterManager() && Availability.NODE.equals(avail)) { + return clusterManager.applyRequest(HttpMethod.DELETE, getAbsolutePath(), getRequestParameters(true), getHeaders()).getResponse(); + } + + // handle expects request (usually from the cluster manager) + final String expects = httpServletRequest.getHeader(WebClusterManager.NCM_EXPECTS_HTTP_HEADER); + if (expects != null) { + serviceFacade.verifyDeleteReportingTask(id); + return generateContinueResponse().build(); + } + + // determine the specified version + Long clientVersion = null; + if (version != null) { + clientVersion = version.getLong(); + } + + // delete the specified reporting task + final ConfigurationSnapshot controllerResponse = serviceFacade.deleteReportingTask(new Revision(clientVersion, clientId.getClientId()), id); + + // get the updated revision + final RevisionDTO revision = new RevisionDTO(); + revision.setClientId(clientId.getClientId()); + revision.setVersion(controllerResponse.getVersion()); + + // build the response entity + final ReportingTaskEntity entity = new ReportingTaskEntity(); + entity.setRevision(revision); + + return clusterContext(generateOkResponse(entity)).build(); + } + + // setters + + public void setServiceFacade(NiFiServiceFacade serviceFacade) { + this.serviceFacade = serviceFacade; + } + + public void setClusterManager(WebClusterManager clusterManager) { + this.clusterManager = clusterManager; + } + + public void setProperties(NiFiProperties properties) { + this.properties = properties; + } +} diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/SnippetResource.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/SnippetResource.java index e0b7788221..275b133a86 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/SnippetResource.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/SnippetResource.java @@ -340,7 +340,7 @@ public class SnippetResource extends ApplicationResource { // get the updated revision final RevisionDTO updatedRevision = new RevisionDTO(); updatedRevision.setClientId(revision.getClientId()); - updatedRevision.setVersion(response.getRevision()); + updatedRevision.setVersion(response.getVersion()); // build the response entity SnippetEntity entity = new SnippetEntity(); @@ -520,7 +520,7 @@ public class SnippetResource extends ApplicationResource { // get the updated revision final RevisionDTO updatedRevision = new RevisionDTO(); updatedRevision.setClientId(revision.getClientId()); - updatedRevision.setVersion(controllerResponse.getRevision()); + updatedRevision.setVersion(controllerResponse.getVersion()); // build the response entity SnippetEntity entity = new SnippetEntity(); @@ -577,7 +577,7 @@ public class SnippetResource extends ApplicationResource { // get the updated revision final RevisionDTO revision = new RevisionDTO(); revision.setClientId(clientId.getClientId()); - revision.setVersion(controllerResponse.getRevision()); + revision.setVersion(controllerResponse.getVersion()); // build the response entity SnippetEntity entity = new SnippetEntity(); diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/config/ThrowableMapper.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/config/ThrowableMapper.java index 091653a4cb..0ef6edbb73 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/config/ThrowableMapper.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/config/ThrowableMapper.java @@ -34,11 +34,8 @@ public class ThrowableMapper implements ExceptionMapper { @Override public Response toResponse(Throwable exception) { // log the error - logger.info(String.format("An unexpected error has occurred: %s. Returning %s response.", exception, Response.Status.INTERNAL_SERVER_ERROR)); - - if (logger.isDebugEnabled()) { - logger.debug(StringUtils.EMPTY, exception); - } + logger.error(String.format("An unexpected error has occurred: %s. Returning %s response.", exception, Response.Status.INTERNAL_SERVER_ERROR)); + logger.error(StringUtils.EMPTY, exception); return Response.status(Response.Status.INTERNAL_SERVER_ERROR).entity("An unexpected error has occurred. Please check the logs for additional details.").type("text/plain").build(); } diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/DtoFactory.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/DtoFactory.java index 92a54499d1..7fe76ad7ae 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/DtoFactory.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/DtoFactory.java @@ -39,7 +39,7 @@ import javax.ws.rs.WebApplicationException; import org.apache.nifi.action.Action; import org.apache.nifi.action.component.details.ComponentDetails; -import org.apache.nifi.action.component.details.ProcessorDetails; +import org.apache.nifi.action.component.details.ExtensionDetails; import org.apache.nifi.action.component.details.RemoteProcessGroupDetails; import org.apache.nifi.action.details.ActionDetails; import org.apache.nifi.action.details.ConfigureDetails; @@ -97,14 +97,12 @@ import org.apache.nifi.scheduling.SchedulingStrategy; import org.apache.nifi.user.NiFiUser; import org.apache.nifi.user.NiFiUserGroup; import org.apache.nifi.util.FormatUtils; -import org.apache.nifi.util.NiFiProperties; import org.apache.nifi.web.Revision; -import org.apache.nifi.web.api.dto.ProcessorConfigDTO.AllowableValueDTO; -import org.apache.nifi.web.api.dto.ProcessorConfigDTO.PropertyDescriptorDTO; +import org.apache.nifi.web.api.dto.PropertyDescriptorDTO.AllowableValueDTO; import org.apache.nifi.web.api.dto.action.ActionDTO; import org.apache.nifi.web.api.dto.action.HistoryDTO; import org.apache.nifi.web.api.dto.action.component.details.ComponentDetailsDTO; -import org.apache.nifi.web.api.dto.action.component.details.ProcessorDetailsDTO; +import org.apache.nifi.web.api.dto.action.component.details.ExtensionDetailsDTO; import org.apache.nifi.web.api.dto.action.component.details.RemoteProcessGroupDetailsDTO; import org.apache.nifi.web.api.dto.action.details.ActionDetailsDTO; import org.apache.nifi.web.api.dto.action.details.ConfigureDetailsDTO; @@ -124,6 +122,12 @@ import org.apache.nifi.web.api.dto.status.ProcessorStatusDTO; import org.apache.nifi.web.api.dto.status.RemoteProcessGroupStatusDTO; import org.apache.nifi.web.api.dto.status.StatusDTO; import org.apache.commons.lang3.StringUtils; +import org.apache.nifi.controller.ConfiguredComponent; +import org.apache.nifi.controller.ReportingTaskNode; +import org.apache.nifi.controller.service.ControllerServiceNode; +import org.apache.nifi.controller.service.ControllerServiceReference; +import org.apache.nifi.reporting.ReportingTask; +import org.apache.nifi.web.FlowModification; /** * @@ -218,9 +222,9 @@ public final class DtoFactory { return null; } - if (componentDetails instanceof ProcessorDetails) { - final ProcessorDetailsDTO processorDetails = new ProcessorDetailsDTO(); - processorDetails.setType(((ProcessorDetails) componentDetails).getType()); + if (componentDetails instanceof ExtensionDetails) { + final ExtensionDetailsDTO processorDetails = new ExtensionDetailsDTO(); + processorDetails.setType(((ExtensionDetails) componentDetails).getType()); return processorDetails; } else if (componentDetails instanceof RemoteProcessGroupDetails) { final RemoteProcessGroupDetailsDTO remoteProcessGroupDetails = new RemoteProcessGroupDetailsDTO(); @@ -834,6 +838,241 @@ public final class DtoFactory { return dto; } + public ReportingTaskDTO createReportingTaskDto(final ReportingTaskNode reportingTaskNode) { + final ReportingTaskDTO dto = new ReportingTaskDTO(); + dto.setId(reportingTaskNode.getIdentifier()); + dto.setName(reportingTaskNode.getName()); + dto.setType(reportingTaskNode.getReportingTask().getClass().getName()); + dto.setSchedulingStrategy(reportingTaskNode.getSchedulingStrategy().name()); + dto.setSchedulingPeriod(reportingTaskNode.getSchedulingPeriod()); + dto.setState(reportingTaskNode.getScheduledState().name()); + dto.setActiveThreadCount(reportingTaskNode.getActiveThreadCount()); + dto.setAnnotationData(reportingTaskNode.getAnnotationData()); + dto.setComments(reportingTaskNode.getComments()); + + final Map defaultSchedulingPeriod = new HashMap<>(); + defaultSchedulingPeriod.put(SchedulingStrategy.TIMER_DRIVEN.name(), SchedulingStrategy.TIMER_DRIVEN.getDefaultSchedulingPeriod()); + defaultSchedulingPeriod.put(SchedulingStrategy.CRON_DRIVEN.name(), SchedulingStrategy.CRON_DRIVEN.getDefaultSchedulingPeriod()); + dto.setDefaultSchedulingPeriod(defaultSchedulingPeriod); + + // sort a copy of the properties + final Map sortedProperties = new TreeMap<>(new Comparator() { + @Override + public int compare(PropertyDescriptor o1, PropertyDescriptor o2) { + return Collator.getInstance(Locale.US).compare(o1.getName(), o2.getName()); + } + }); + sortedProperties.putAll(reportingTaskNode.getProperties()); + + // get the property order from the reporting task + final ReportingTask reportingTask = reportingTaskNode.getReportingTask(); + final Map orderedProperties = new LinkedHashMap<>(); + final List descriptors = reportingTask.getPropertyDescriptors(); + if (descriptors != null && !descriptors.isEmpty()) { + for (PropertyDescriptor descriptor : descriptors) { + orderedProperties.put(descriptor, null); + } + } + orderedProperties.putAll(sortedProperties); + + // build the descriptor and property dtos + dto.setDescriptors(new LinkedHashMap()); + dto.setProperties(new LinkedHashMap()); + for (final Map.Entry entry : orderedProperties.entrySet()) { + final PropertyDescriptor descriptor = entry.getKey(); + + // store the property descriptor + dto.getDescriptors().put(descriptor.getName(), createPropertyDescriptorDto(descriptor)); + + // determine the property value - don't include sensitive properties + String propertyValue = entry.getValue(); + if (propertyValue != null && descriptor.isSensitive()) { + propertyValue = "********"; + } + + // set the property value + dto.getProperties().put(descriptor.getName(), propertyValue); + } + + // add the validation errors + final Collection validationErrors = reportingTaskNode.getValidationErrors(); + if (validationErrors != null && !validationErrors.isEmpty()) { + final List errors = new ArrayList<>(); + for (final ValidationResult validationResult : validationErrors) { + errors.add(validationResult.toString()); + } + + dto.setValidationErrors(errors); + } + + return dto; + } + + public ControllerServiceDTO createControllerServiceDto(final ControllerServiceNode controllerServiceNode) { + final ControllerServiceDTO dto = new ControllerServiceDTO(); + dto.setId(controllerServiceNode.getIdentifier()); + dto.setName(controllerServiceNode.getName()); + dto.setType(controllerServiceNode.getControllerServiceImplementation().getClass().getName()); + dto.setState(controllerServiceNode.getState().name()); + dto.setAnnotationData(controllerServiceNode.getAnnotationData()); + dto.setComments(controllerServiceNode.getComments()); + + // sort a copy of the properties + final Map sortedProperties = new TreeMap<>(new Comparator() { + @Override + public int compare(PropertyDescriptor o1, PropertyDescriptor o2) { + return Collator.getInstance(Locale.US).compare(o1.getName(), o2.getName()); + } + }); + sortedProperties.putAll(controllerServiceNode.getProperties()); + + // get the property order from the controller service + final ControllerService controllerService = controllerServiceNode.getControllerServiceImplementation(); + final Map orderedProperties = new LinkedHashMap<>(); + final List descriptors = controllerService.getPropertyDescriptors(); + if (descriptors != null && !descriptors.isEmpty()) { + for (PropertyDescriptor descriptor : descriptors) { + orderedProperties.put(descriptor, null); + } + } + orderedProperties.putAll(sortedProperties); + + // build the descriptor and property dtos + dto.setDescriptors(new LinkedHashMap()); + dto.setProperties(new LinkedHashMap()); + for (final Map.Entry entry : orderedProperties.entrySet()) { + final PropertyDescriptor descriptor = entry.getKey(); + + // store the property descriptor + dto.getDescriptors().put(descriptor.getName(), createPropertyDescriptorDto(descriptor)); + + // determine the property value - don't include sensitive properties + String propertyValue = entry.getValue(); + if (propertyValue != null && descriptor.isSensitive()) { + propertyValue = "********"; + } + + // set the property value + dto.getProperties().put(descriptor.getName(), propertyValue); + } + + // create the reference dto's + dto.setReferencingComponents(createControllerServiceReferencingComponentsDto(controllerServiceNode.getReferences())); + + // add the validation errors + final Collection validationErrors = controllerServiceNode.getValidationErrors(); + if (validationErrors != null && !validationErrors.isEmpty()) { + final List errors = new ArrayList<>(); + for (final ValidationResult validationResult : validationErrors) { + errors.add(validationResult.toString()); + } + + dto.setValidationErrors(errors); + } + + return dto; + } + + public Set createControllerServiceReferencingComponentsDto(final ControllerServiceReference reference) { + return createControllerServiceReferencingComponentsDto(reference, new HashSet()); + } + + private Set createControllerServiceReferencingComponentsDto(final ControllerServiceReference reference, final Set visited) { + final Set referencingComponents = new LinkedHashSet<>(); + + // get all references + for (final ConfiguredComponent component : reference.getReferencingComponents()) { + final ControllerServiceReferencingComponentDTO dto = new ControllerServiceReferencingComponentDTO(); + dto.setId(component.getIdentifier()); + dto.setName(component.getName()); + + List propertyDescriptors = null; + Collection validationErrors = null; + if (component instanceof ProcessorNode) { + final ProcessorNode node = ((ProcessorNode) component); + dto.setGroupId(node.getProcessGroup().getIdentifier()); + dto.setState(node.getScheduledState().name()); + dto.setActiveThreadCount(node.getActiveThreadCount()); + dto.setType(node.getProcessor().getClass().getName()); + dto.setReferenceType(Processor.class.getSimpleName()); + + propertyDescriptors = node.getProcessor().getPropertyDescriptors(); + validationErrors = node.getValidationErrors(); + } else if (component instanceof ControllerServiceNode) { + final ControllerServiceNode node = ((ControllerServiceNode) component); + dto.setState(node.getState().name()); + dto.setType(node.getControllerServiceImplementation().getClass().getName()); + dto.setReferenceType(ControllerService.class.getSimpleName()); + dto.setReferenceCycle(visited.contains(node)); + + // if we haven't encountered this service before include it's referencing components + if (!dto.getReferenceCycle()) { + dto.setReferencingComponents(createControllerServiceReferencingComponentsDto(node.getReferences(), visited)); + } + + propertyDescriptors = node.getControllerServiceImplementation().getPropertyDescriptors(); + validationErrors = node.getValidationErrors(); + } else if (component instanceof ReportingTaskNode) { + final ReportingTaskNode node = ((ReportingTaskNode) component); + dto.setState(node.getScheduledState().name()); + dto.setActiveThreadCount(node.getActiveThreadCount()); + dto.setType(node.getReportingTask().getClass().getName()); + dto.setReferenceType(ReportingTask.class.getSimpleName()); + + propertyDescriptors = node.getReportingTask().getPropertyDescriptors(); + validationErrors = node.getValidationErrors(); + } + + if (propertyDescriptors != null && !propertyDescriptors.isEmpty()) { + final Map sortedProperties = new TreeMap<>(new Comparator() { + @Override + public int compare(PropertyDescriptor o1, PropertyDescriptor o2) { + return Collator.getInstance(Locale.US).compare(o1.getName(), o2.getName()); + } + }); + sortedProperties.putAll(component.getProperties()); + + final Map orderedProperties = new LinkedHashMap<>(); + for (PropertyDescriptor descriptor : propertyDescriptors) { + orderedProperties.put(descriptor, null); + } + orderedProperties.putAll(sortedProperties); + + // build the descriptor and property dtos + dto.setDescriptors(new LinkedHashMap()); + dto.setProperties(new LinkedHashMap()); + for (final Map.Entry entry : orderedProperties.entrySet()) { + final PropertyDescriptor descriptor = entry.getKey(); + + // store the property descriptor + dto.getDescriptors().put(descriptor.getName(), createPropertyDescriptorDto(descriptor)); + + // determine the property value - don't include sensitive properties + String propertyValue = entry.getValue(); + if (propertyValue != null && descriptor.isSensitive()) { + propertyValue = "********"; + } + + // set the property value + dto.getProperties().put(descriptor.getName(), propertyValue); + } + } + + if (validationErrors != null && !validationErrors.isEmpty()) { + final List errors = new ArrayList<>(); + for (final ValidationResult validationResult : validationErrors) { + errors.add(validationResult.toString()); + } + + dto.setValidationErrors(errors); + } + + referencingComponents.add(dto); + } + + return referencingComponents; + } + public RemoteProcessGroupPortDTO createRemoteProcessGroupPortDto(final RemoteGroupPort port) { if (port == null) { return null; @@ -1143,6 +1382,135 @@ public final class DtoFactory { return types; } + + /** + * Identifies all baseTypes for the specified type that are assignable to the specified baseType. + * + * @param baseType + * @param type + * @param baseTypes + */ + private void identifyBaseTypes(final Class baseType, final Class type, final Set baseTypes, final boolean recurse) { + final Class[] interfaces = type.getInterfaces(); + for (final Class i : interfaces) { + if (baseType.isAssignableFrom(i) && !baseType.equals(i)) { + baseTypes.add(i); + } + } + + if (recurse) { + if (type.getSuperclass() != null) { + identifyBaseTypes(baseType, type.getSuperclass(), baseTypes, recurse); + } + } + } + + /** + * Gets the DocumentedTypeDTOs from the specified classes for the specified baseClass. + * + * @param baseClass + * @param classes + * @return + */ + public Set fromDocumentedTypes(final Class baseClass, final Set classes) { + final Set types = new LinkedHashSet<>(); + final Set sortedClasses = new TreeSet<>(CLASS_NAME_COMPARATOR); + sortedClasses.addAll(classes); + + // identify all interfaces that extend baseClass for all classes + final Set interfaces = new HashSet<>(); + for (final Class cls : sortedClasses) { + identifyBaseTypes(baseClass, cls, interfaces, true); + } + + // build a lookup of all interfaces + final Map lookup = new HashMap<>(); + + // convert the interfaces to DTO form + for (final Class i : interfaces) { + final DocumentedTypeDTO type = new DocumentedTypeDTO(); + type.setType(i.getName()); + type.setDescription(getCapabilityDescription(i)); + type.setTags(getTags(i)); + type.setChildTypes(new LinkedHashSet()); + lookup.put(i, type); + } + + // move the interfaces into the appropriate hierarchy + final Collection rootTypes = new ArrayList<>(); + for (final Class i : interfaces) { + rootTypes.add(i); + + // identify the base types + final Set baseTypes = new LinkedHashSet<>(); + identifyBaseTypes(baseClass, i, baseTypes, false); + + // move this interfaces into the hierarchy where appropriate + if (!baseTypes.isEmpty()) { + // get the DTO for each base type + for (final Class baseType : baseTypes) { + final DocumentedTypeDTO parentInteface = lookup.get(baseType); + final DocumentedTypeDTO childInterface = lookup.get(i); + + // include all parent tags in the respective children + childInterface.getTags().addAll(parentInteface.getTags()); + + // update the hierarchy + parentInteface.getChildTypes().add(childInterface); + } + + // remove this interface from the lookup (this will only + // leave the interfaces that are ancestor roots) + rootTypes.remove(i); + } + } + + // include the interfaces + sortedClasses.addAll(rootTypes); + + // get the DTO form for all interfaces and classes + for (final Class cls : sortedClasses) { + boolean add = false; + + final DocumentedTypeDTO type; + if (rootTypes.contains(cls)) { + type = lookup.get(cls); + add = true; + } else { + type = new DocumentedTypeDTO(); + type.setType(cls.getName()); + type.setDescription(getCapabilityDescription(cls)); + type.setTags(getTags(cls)); + } + + // identify the base types + final Set baseTypes = new LinkedHashSet<>(); + identifyBaseTypes(baseClass, cls, baseTypes, false); + + // include this type if it doesn't belong to another hierarchy + if (baseTypes.isEmpty()) { + add = true; + } else { + // get the DTO for each base type + for (final Class baseType : baseTypes) { + final DocumentedTypeDTO parentInterface = lookup.get(baseType); + + // include all parent tags in the respective children + type.getTags().addAll(parentInterface.getTags()); + + // update the hierarchy + parentInterface.getChildTypes().add(type); + } + } + + // add if appropriate + if (add) { + types.add(type); + } + } + + return types; + } /** * Creates a ProcessorDTO from the specified ProcessorNode. @@ -1575,12 +1943,12 @@ public final class DtoFactory { * @param propertyDescriptor * @return */ - private ProcessorConfigDTO.PropertyDescriptorDTO createPropertyDescriptorDto(final PropertyDescriptor propertyDescriptor) { + public PropertyDescriptorDTO createPropertyDescriptorDto(final PropertyDescriptor propertyDescriptor) { if (propertyDescriptor == null) { return null; } - final ProcessorConfigDTO.PropertyDescriptorDTO dto = new ProcessorConfigDTO.PropertyDescriptorDTO(); + final PropertyDescriptorDTO dto = new PropertyDescriptorDTO(); dto.setName(propertyDescriptor.getName()); dto.setDisplayName(propertyDescriptor.getDisplayName()); @@ -1590,18 +1958,16 @@ public final class DtoFactory { dto.setDescription(propertyDescriptor.getDescription()); dto.setDefaultValue(propertyDescriptor.getDefaultValue()); dto.setSupportsEl(propertyDescriptor.isExpressionLanguageSupported()); + dto.setIdentifiesControllerService(propertyDescriptor.getControllerServiceDefinition() != null); final Class serviceDefinition = propertyDescriptor.getControllerServiceDefinition(); if (propertyDescriptor.getAllowableValues() == null) { if (serviceDefinition == null) { dto.setAllowableValues(null); } else { - final Set allowableValues = new LinkedHashSet<>(); + final List allowableValues = new ArrayList<>(); for (final String serviceIdentifier : controllerServiceLookup.getControllerServiceIdentifiers(serviceDefinition)) { - String displayName = serviceIdentifier; - - // TODO: attempt to get the controller service name - final ControllerService controllerService = controllerServiceLookup.getControllerService(serviceIdentifier); + final String displayName = controllerServiceLookup.getControllerServiceName(serviceIdentifier); final AllowableValueDTO allowableValue = new AllowableValueDTO(); allowableValue.setDisplayName(displayName); @@ -1611,7 +1977,7 @@ public final class DtoFactory { dto.setAllowableValues(allowableValues); } } else { - final Set allowableValues = new LinkedHashSet<>(); + final List allowableValues = new ArrayList<>(); for (final AllowableValue allowableValue : propertyDescriptor.getAllowableValues()) { final AllowableValueDTO allowableValueDto = new AllowableValueDTO(); allowableValueDto.setDisplayName(allowableValue.getDisplayName()); @@ -1642,6 +2008,25 @@ public final class DtoFactory { return copy; } + + public ControllerServiceDTO copy(final ControllerServiceDTO original) { + final ControllerServiceDTO copy = new ControllerServiceDTO(); + copy.setAnnotationData(original.getAnnotationData()); + copy.setAvailability(original.getAvailability()); + copy.setComments(original.getComments()); + copy.setCustomUiUrl(original.getCustomUiUrl()); + copy.setDescriptors(copy(original.getDescriptors())); + copy.setId(original.getId()); + copy.setName(original.getName()); + copy.setProperties(copy(original.getProperties())); + copy.setReferencingComponents(copy(original.getReferencingComponents())); + copy.setState(original.getState()); + copy.setType(original.getType()); + copy.setUri(original.getUri()); + copy.setValidationErrors(copy(original.getValidationErrors())); + return copy; + } + public FunnelDTO copy(final FunnelDTO original) { final FunnelDTO copy = new FunnelDTO(); copy.setId(original.getId()); @@ -2029,14 +2414,17 @@ public final class DtoFactory { /** * Factory method for creating a new RevisionDTO based on this controller. * - * @param revision + * @param lastMod * @return */ - public RevisionDTO createRevisionDTO(Revision revision) { + public RevisionDTO createRevisionDTO(FlowModification lastMod) { + final Revision revision = lastMod.getRevision(); + // create the dto final RevisionDTO revisionDTO = new RevisionDTO(); revisionDTO.setVersion(revision.getVersion()); revisionDTO.setClientId(revision.getClientId()); + revisionDTO.setLastModifier(lastMod.getLastModifier()); return revisionDTO; } @@ -2146,8 +2534,6 @@ public final class DtoFactory { } /* setters */ - public void setProperties(NiFiProperties properties) { - } public void setControllerServiceLookup(ControllerServiceLookup lookup) { this.controllerServiceLookup = lookup; diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerFacade.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerFacade.java index 117555a262..a373f05209 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerFacade.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerFacade.java @@ -52,8 +52,6 @@ import org.apache.nifi.controller.ProcessorNode; import org.apache.nifi.controller.ScheduledState; import org.apache.nifi.controller.repository.ContentNotFoundException; import org.apache.nifi.controller.repository.claim.ContentDirection; -import org.apache.nifi.controller.service.ControllerServiceNode; -import org.apache.nifi.controller.service.ControllerServiceProvider; import org.apache.nifi.controller.status.ProcessGroupStatus; import org.apache.nifi.diagnostics.SystemDiagnostics; import org.apache.nifi.flowfile.FlowFilePrioritizer; @@ -79,11 +77,11 @@ import org.apache.nifi.provenance.search.SearchableField; import org.apache.nifi.remote.RootGroupPort; import org.apache.nifi.reporting.Bulletin; import org.apache.nifi.reporting.BulletinRepository; +import org.apache.nifi.reporting.ReportingTask; import org.apache.nifi.scheduling.SchedulingStrategy; import org.apache.nifi.search.SearchContext; import org.apache.nifi.search.SearchResult; import org.apache.nifi.search.Searchable; -import org.apache.nifi.web.security.user.NiFiUserUtils; import org.apache.nifi.services.FlowService; import org.apache.nifi.user.NiFiUser; import org.apache.nifi.util.FormatUtils; @@ -114,6 +112,7 @@ import org.apache.commons.lang3.StringUtils; import org.apache.nifi.admin.service.UserService; import org.apache.nifi.authorization.DownloadAuthorization; import org.apache.nifi.processor.DataUnit; +import org.apache.nifi.web.security.user.NiFiUserUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.springframework.security.access.AccessDeniedException; @@ -121,7 +120,7 @@ import org.springframework.security.access.AccessDeniedException; /** * */ -public class ControllerFacade implements ControllerServiceProvider { +public class ControllerFacade { private static final Logger logger = LoggerFactory.getLogger(ControllerFacade.class); @@ -347,6 +346,24 @@ public class ControllerFacade implements ControllerServiceProvider { public Set getFlowFileComparatorTypes() { return dtoFactory.fromDocumentedTypes(ExtensionManager.getExtensions(FlowFilePrioritizer.class)); } + + /** + * Gets the ControllerService types that this controller supports. + * + * @return + */ + public Set getControllerServiceTypes() { + return dtoFactory.fromDocumentedTypes(ControllerService.class, ExtensionManager.getExtensions(ControllerService.class)); + } + + /** + * Gets the ReportingTask types that this controller supports. + * + * @return + */ + public Set getReportingTaskTypes() { + return dtoFactory.fromDocumentedTypes(ReportingTask.class, ExtensionManager.getExtensions(ReportingTask.class)); + } /** * Gets the counters for this controller. @@ -371,56 +388,6 @@ public class ControllerFacade implements ControllerServiceProvider { return counter; } - - /** - * Return the controller service for the specified identifier. - * - * @param serviceIdentifier - * @return - */ - @Override - public ControllerService getControllerService(String serviceIdentifier) { - return flowController.getControllerService(serviceIdentifier); - } - - @Override - public ControllerServiceNode createControllerService(String type, String id, boolean firstTimeAdded) { - return flowController.createControllerService(type, id, firstTimeAdded); - } - - public void removeControllerService(ControllerServiceNode serviceNode) { - flowController.removeControllerService(serviceNode); - } - - @Override - public Set getControllerServiceIdentifiers(Class serviceType) { - return flowController.getControllerServiceIdentifiers(serviceType); - } - - @Override - public ControllerServiceNode getControllerServiceNode(final String id) { - return flowController.getControllerServiceNode(id); - } - - @Override - public boolean isControllerServiceEnabled(final ControllerService service) { - return flowController.isControllerServiceEnabled(service); - } - - @Override - public boolean isControllerServiceEnabled(final String serviceIdentifier) { - return flowController.isControllerServiceEnabled(serviceIdentifier); - } - - @Override - public void enableControllerService(final ControllerServiceNode serviceNode) { - flowController.enableControllerService(serviceNode); - } - - @Override - public void disableControllerService(ControllerServiceNode serviceNode) { - flowController.disableControllerService(serviceNode); - } /** * Gets the status of this controller. diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/ControllerServiceDAO.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/ControllerServiceDAO.java new file mode 100644 index 0000000000..c1fba0c6b5 --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/ControllerServiceDAO.java @@ -0,0 +1,110 @@ +/* + * 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.web.dao; + +import java.util.Set; +import org.apache.nifi.controller.ScheduledState; + +import org.apache.nifi.controller.service.ControllerServiceNode; +import org.apache.nifi.controller.service.ControllerServiceReference; +import org.apache.nifi.controller.service.ControllerServiceState; +import org.apache.nifi.web.api.dto.ControllerServiceDTO; + +/** + * + */ +public interface ControllerServiceDAO { + + /** + * Determines if the specified controller service exists. + * + * @param controllerServiceId + * @return + */ + boolean hasControllerService(String controllerServiceId); + + /** + * Creates a controller service. + * + * @param controllerServiceDTO The controller service DTO + * @return The controller service + */ + ControllerServiceNode createControllerService(ControllerServiceDTO controllerServiceDTO); + + /** + * Gets the specified controller service. + * + * @param controllerServiceId The controller service id + * @return The controller service + */ + ControllerServiceNode getControllerService(String controllerServiceId); + + /** + * Gets all of the controller services. + * + * @return The controller services + */ + Set getControllerServices(); + + /** + * Updates the specified controller service. + * + * @param controllerServiceDTO The controller service DTO + * @return The controller service + */ + ControllerServiceNode updateControllerService(ControllerServiceDTO controllerServiceDTO); + + /** + * Updates the referencing components for the specified controller service. + * + * @param controllerServiceId + * @param scheduledState + * @param controllerServiceState the value of state + * @return the org.apache.nifi.controller.service.ControllerServiceReference + */ + ControllerServiceReference updateControllerServiceReferencingComponents(String controllerServiceId, ScheduledState scheduledState, ControllerServiceState controllerServiceState); + + /** + * Determines whether this controller service can be updated. + * + * @param controllerServiceDTO + */ + void verifyUpdate(ControllerServiceDTO controllerServiceDTO); + + /** + * Determines whether the referencing component of the specified controller service can be updated. + * + * @param controllerServiceId + * @param scheduledState + * @param controllerServiceState + */ + void verifyUpdateReferencingComponents(String controllerServiceId, ScheduledState scheduledState, ControllerServiceState controllerServiceState); + + /** + * Determines whether this controller service can be removed. + * + * @param controllerServiceId + */ + void verifyDelete(String controllerServiceId); + + /** + * Deletes the specified controller service. + * + * @param controllerServiceId The controller service id + */ + void deleteControllerService(String controllerServiceId); +} diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/ReportingTaskDAO.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/ReportingTaskDAO.java new file mode 100644 index 0000000000..49446d396e --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/ReportingTaskDAO.java @@ -0,0 +1,88 @@ +/* + * 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.web.dao; + +import java.util.Set; +import org.apache.nifi.controller.ReportingTaskNode; + +import org.apache.nifi.web.api.dto.ReportingTaskDTO; + +/** + * + */ +public interface ReportingTaskDAO { + + /** + * Determines if the specified reporting task exists. + * + * @param reportingTaskId + * @return + */ + boolean hasReportingTask(String reportingTaskId); + + /** + * Creates a reporting task. + * + * @param reportingTaskDTO The reporting task DTO + * @return The reporting task + */ + ReportingTaskNode createReportingTask(ReportingTaskDTO reportingTaskDTO); + + /** + * Gets the specified reporting task. + * + * @param reportingTaskId The reporting task id + * @return The reporting task + */ + ReportingTaskNode getReportingTask(String reportingTaskId); + + /** + * Gets all of the reporting tasks. + * + * @return The reporting tasks + */ + Set getReportingTasks(); + + /** + * Updates the specified reporting task. + * + * @param reportingTaskDTO The reporting task DTO + * @return The reporting task + */ + ReportingTaskNode updateReportingTask(ReportingTaskDTO reportingTaskDTO); + + /** + * Determines whether this reporting task can be updated. + * + * @param reportingTaskDTO + */ + void verifyUpdate(ReportingTaskDTO reportingTaskDTO); + + /** + * Determines whether this reporting task can be removed. + * + * @param reportingTaskId + */ + void verifyDelete(String reportingTaskId); + + /** + * Deletes the specified reporting task. + * + * @param reportingTaskId The reporting task id + */ + void deleteReportingTask(String reportingTaskId); +} diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardControllerServiceDAO.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardControllerServiceDAO.java new file mode 100644 index 0000000000..14217c583f --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardControllerServiceDAO.java @@ -0,0 +1,320 @@ +/* + * 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.web.dao.impl; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.Set; +import org.apache.nifi.controller.ScheduledState; +import org.apache.nifi.controller.exception.ControllerServiceInstantiationException; + +import org.apache.nifi.controller.exception.ValidationException; +import org.apache.nifi.controller.service.ControllerServiceNode; +import org.apache.nifi.controller.service.ControllerServiceProvider; +import org.apache.nifi.controller.service.ControllerServiceReference; +import org.apache.nifi.controller.service.ControllerServiceState; +import org.apache.nifi.web.NiFiCoreException; +import org.apache.nifi.web.ResourceNotFoundException; +import org.apache.nifi.web.api.dto.ControllerServiceDTO; +import org.apache.nifi.web.dao.ControllerServiceDAO; + +public class StandardControllerServiceDAO extends ComponentDAO implements ControllerServiceDAO { + + private ControllerServiceProvider serviceProvider; + + /** + * Locates the specified controller service. + * + * @param controllerServiceId + * @return + */ + private ControllerServiceNode locateControllerService(final String controllerServiceId) { + // get the controller service + final ControllerServiceNode controllerService = serviceProvider.getControllerServiceNode(controllerServiceId); + + // ensure the controller service exists + if (controllerService == null) { + throw new ResourceNotFoundException(String.format("Unable to locate controller service with id '%s'.", controllerServiceId)); + } + + return controllerService; + } + + /** + * Creates a controller service. + * + * @param controllerServiceDTO The controller service DTO + * @return The controller service + */ + @Override + public ControllerServiceNode createControllerService(final ControllerServiceDTO controllerServiceDTO) { + // ensure the type is specified + if (controllerServiceDTO.getType() == null) { + throw new IllegalArgumentException("The controller service type must be specified."); + } + + try { + // create the controller service + final ControllerServiceNode controllerService = serviceProvider.createControllerService(controllerServiceDTO.getType(), controllerServiceDTO.getId(), true); + + // ensure we can perform the update + verifyUpdate(controllerService, controllerServiceDTO); + + // perform the update + configureControllerService(controllerService, controllerServiceDTO); + + return controllerService; + } catch (final ControllerServiceInstantiationException csie) { + throw new NiFiCoreException(csie.getMessage(), csie); + } + } + + /** + * Gets the specified controller service. + * + * @param controllerServiceId The controller service id + * @return The controller service + */ + @Override + public ControllerServiceNode getControllerService(final String controllerServiceId) { + return locateControllerService(controllerServiceId); + } + + /** + * Determines if the specified controller service exists. + * + * @param controllerServiceId + * @return + */ + @Override + public boolean hasControllerService(final String controllerServiceId) { + return serviceProvider.getControllerServiceNode(controllerServiceId) != null; + } + + /** + * Gets all of the controller services. + * + * @return The controller services + */ + @Override + public Set getControllerServices() { + return serviceProvider.getAllControllerServices(); + } + + /** + * Updates the specified controller service. + * + * @param controllerServiceDTO The controller service DTO + * @return The controller service + */ + @Override + public ControllerServiceNode updateControllerService(final ControllerServiceDTO controllerServiceDTO) { + // get the controller service + final ControllerServiceNode controllerService = locateControllerService(controllerServiceDTO.getId()); + + // ensure we can perform the update + verifyUpdate(controllerService, controllerServiceDTO); + + // perform the update + configureControllerService(controllerService, controllerServiceDTO); + + // enable or disable as appropriate + if (isNotNull(controllerServiceDTO.getState())) { + final ControllerServiceState purposedControllerServiceState = ControllerServiceState.valueOf(controllerServiceDTO.getState()); + + // only attempt an action if it is changing + if (!purposedControllerServiceState.equals(controllerService.getState())) { + if (ControllerServiceState.ENABLED.equals(purposedControllerServiceState)) { + serviceProvider.enableControllerService(controllerService); + } else if (ControllerServiceState.DISABLED.equals(purposedControllerServiceState)) { + serviceProvider.disableControllerService(controllerService); + } + } + } + + return controllerService; + } + + @Override + public ControllerServiceReference updateControllerServiceReferencingComponents(final String controllerServiceId, final ScheduledState scheduledState, final ControllerServiceState controllerServiceState) { + // get the controller service + final ControllerServiceNode controllerService = locateControllerService(controllerServiceId); + + // this request is either acting upon referncing services or schedulable components + if (controllerServiceState != null) { + if (ControllerServiceState.ENABLED.equals(controllerServiceState)) { + serviceProvider.enableReferencingServices(controllerService); + } else { + serviceProvider.disableReferencingServices(controllerService); + } + } else if (scheduledState != null) { + if (ScheduledState.RUNNING.equals(scheduledState)) { + serviceProvider.scheduleReferencingComponents(controllerService); + } else { + serviceProvider.unscheduleReferencingComponents(controllerService); + } + } + + return controllerService.getReferences(); + } + + /** + * Validates the specified configuration for the specified controller service. + * + * @param controllerService + * @param controllerServiceDTO + * @return + */ + private List validateProposedConfiguration(final ControllerServiceNode controllerService, final ControllerServiceDTO controllerServiceDTO) { + final List validationErrors = new ArrayList<>(); + return validationErrors; + } + + @Override + public void verifyDelete(final String controllerServiceId) { + final ControllerServiceNode controllerService = locateControllerService(controllerServiceId); + controllerService.verifyCanDelete(); + } + + @Override + public void verifyUpdate(final ControllerServiceDTO controllerServiceDTO) { + final ControllerServiceNode controllerService = locateControllerService(controllerServiceDTO.getId()); + verifyUpdate(controllerService, controllerServiceDTO); + } + + @Override + public void verifyUpdateReferencingComponents(String controllerServiceId, ScheduledState scheduledState, ControllerServiceState controllerServiceState) { + final ControllerServiceNode controllerService = locateControllerService(controllerServiceId); + + if (controllerServiceState != null) { + if (ControllerServiceState.ENABLED.equals(controllerServiceState)) { + serviceProvider.verifyCanEnableReferencingServices(controllerService); + } else { + serviceProvider.verifyCanDisableReferencingServices(controllerService); + } + } else if (scheduledState != null) { + if (ScheduledState.RUNNING.equals(scheduledState)) { + serviceProvider.verifyCanScheduleReferencingComponents(controllerService); + } else { + serviceProvider.verifyCanStopReferencingComponents(controllerService); + } + } + } + + /** + * Verifies the controller service can be updated. + * + * @param controllerService + * @param controllerServiceDTO + */ + private void verifyUpdate(final ControllerServiceNode controllerService, final ControllerServiceDTO controllerServiceDTO) { + // validate the new controller service state if appropriate + if (isNotNull(controllerServiceDTO.getState())) { + try { + // attempt to parse the service state + final ControllerServiceState purposedControllerServiceState = ControllerServiceState.valueOf(controllerServiceDTO.getState()); + + // ensure the state is valid + if (ControllerServiceState.ENABLING.equals(purposedControllerServiceState) || ControllerServiceState.DISABLING.equals(purposedControllerServiceState)) { + throw new IllegalArgumentException(); + } + + // only attempt an action if it is changing + if (!purposedControllerServiceState.equals(controllerService.getState())) { + if (ControllerServiceState.ENABLED.equals(purposedControllerServiceState)) { + controllerService.verifyCanEnable(); + } else if (ControllerServiceState.DISABLED.equals(purposedControllerServiceState)) { + controllerService.verifyCanDisable(); + } + } + } catch (IllegalArgumentException iae) { + throw new IllegalArgumentException("Controller Service state: Value must be one of [ENABLED, DISABLED]"); + } + } + + boolean modificationRequest = false; + if (isAnyNotNull(controllerServiceDTO.getName(), + controllerServiceDTO.getAnnotationData(), + controllerServiceDTO.getComments(), + controllerServiceDTO.getProperties())) { + modificationRequest = true; + + // validate the request + final List requestValidation = validateProposedConfiguration(controllerService, controllerServiceDTO); + + // ensure there was no validation errors + if (!requestValidation.isEmpty()) { + throw new ValidationException(requestValidation); + } + } + + if (modificationRequest) { + controllerService.verifyCanUpdate(); + } + } + + /** + * Configures the specified controller service. + * + * @param controllerService + * @param controllerServiceDTO + */ + private void configureControllerService(final ControllerServiceNode controllerService, final ControllerServiceDTO controllerServiceDTO) { + final String name = controllerServiceDTO.getName(); + final String annotationData = controllerServiceDTO.getAnnotationData(); + final String comments = controllerServiceDTO.getComments(); + final Map properties = controllerServiceDTO.getProperties(); + + if (isNotNull(name)) { + controllerService.setName(name); + } + if (isNotNull(annotationData)) { + controllerService.setAnnotationData(annotationData); + } + if (isNotNull(comments)) { + controllerService.setComments(comments); + } + if (isNotNull(properties)) { + for (final Map.Entry entry : properties.entrySet()) { + final String propName = entry.getKey(); + final String propVal = entry.getValue(); + if (isNotNull(propName) && propVal == null) { + controllerService.removeProperty(propName); + } else if (isNotNull(propName)) { + controllerService.setProperty(propName, propVal); + } + } + } + } + + /** + * Deletes the specified controller service. + * + * @param controllerServiceId The controller service id + */ + @Override + public void deleteControllerService(String controllerServiceId) { + final ControllerServiceNode controllerService = locateControllerService(controllerServiceId); + serviceProvider.removeControllerService(controllerService); + } + + /* setters */ + public void setServiceProvider(ControllerServiceProvider serviceProvider) { + this.serviceProvider = serviceProvider; + } +} diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardProcessorDAO.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardProcessorDAO.java index 633f8e20ed..0c587fe119 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardProcessorDAO.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardProcessorDAO.java @@ -99,6 +99,11 @@ public class StandardProcessorDAO extends ComponentDAO implements ProcessorDAO { if (processorDTO.getParentGroupId() != null && !flowController.areGroupsSame(groupId, processorDTO.getParentGroupId())) { throw new IllegalArgumentException("Cannot specify a different Parent Group ID than the Group to which the Processor is being added."); } + + // ensure the type is specified + if (processorDTO.getType() == null) { + throw new IllegalArgumentException("The processor type must be specified."); + } // get the group to add the processor to ProcessGroup group = locateProcessGroup(flowController, groupId); diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardReportingTaskDAO.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardReportingTaskDAO.java new file mode 100644 index 0000000000..d9fd74c509 --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardReportingTaskDAO.java @@ -0,0 +1,365 @@ +/* + * 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.web.dao.impl; + +import java.text.ParseException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.RejectedExecutionException; +import java.util.regex.Matcher; +import org.apache.commons.lang3.StringUtils; +import org.apache.nifi.controller.ReportingTaskNode; +import org.apache.nifi.controller.ScheduledState; +import org.apache.nifi.controller.exception.ProcessorLifeCycleException; + +import org.apache.nifi.controller.exception.ValidationException; +import org.apache.nifi.controller.reporting.ReportingTaskInstantiationException; +import org.apache.nifi.controller.reporting.ReportingTaskProvider; +import org.apache.nifi.scheduling.SchedulingStrategy; +import org.apache.nifi.util.FormatUtils; +import org.apache.nifi.web.NiFiCoreException; +import org.apache.nifi.web.ResourceNotFoundException; +import org.apache.nifi.web.api.dto.ReportingTaskDTO; +import org.apache.nifi.web.dao.ReportingTaskDAO; +import org.quartz.CronExpression; + +public class StandardReportingTaskDAO extends ComponentDAO implements ReportingTaskDAO { + + private ReportingTaskProvider reportingTaskProvider; + + /** + * Locates the specified reporting task. + * + * @param reportingTaskId + * @return + */ + private ReportingTaskNode locateReportingTask(final String reportingTaskId) { + // get the reporting task + final ReportingTaskNode reportingTask = reportingTaskProvider.getReportingTaskNode(reportingTaskId); + + // ensure the reporting task exists + if (reportingTask == null) { + throw new ResourceNotFoundException(String.format("Unable to locate reporting task with id '%s'.", reportingTaskId)); + } + + return reportingTask; + } + + /** + * Creates a reporting task. + * + * @param reportingTaskDTO The reporting task DTO + * @return The reporting task + */ + @Override + public ReportingTaskNode createReportingTask(final ReportingTaskDTO reportingTaskDTO) { + // ensure the type is specified + if (reportingTaskDTO.getType() == null) { + throw new IllegalArgumentException("The reporting task type must be specified."); + } + + try { + // create the reporting task + final ReportingTaskNode reportingTask = reportingTaskProvider.createReportingTask(reportingTaskDTO.getType(), reportingTaskDTO.getId(), true); + + // ensure we can perform the update + verifyUpdate(reportingTask, reportingTaskDTO); + + // perform the update + configureReportingTask(reportingTask, reportingTaskDTO); + + return reportingTask; + } catch (ReportingTaskInstantiationException rtie) { + throw new NiFiCoreException(rtie.getMessage(), rtie); + } + } + + /** + * Gets the specified reporting task. + * + * @param reportingTaskId The reporting task id + * @return The reporting task + */ + @Override + public ReportingTaskNode getReportingTask(final String reportingTaskId) { + return locateReportingTask(reportingTaskId); + } + + /** + * Determines if the specified reporting task exists. + * + * @param reportingTaskId + * @return + */ + @Override + public boolean hasReportingTask(final String reportingTaskId) { + return reportingTaskProvider.getReportingTaskNode(reportingTaskId) != null; + } + + /** + * Gets all of the reporting tasks. + * + * @return The reporting tasks + */ + @Override + public Set getReportingTasks() { + return reportingTaskProvider.getAllReportingTasks(); + } + + /** + * Updates the specified reporting task. + * + * @param reportingTaskDTO The reporting task DTO + * @return The reporting task + */ + @Override + public ReportingTaskNode updateReportingTask(final ReportingTaskDTO reportingTaskDTO) { + // get the reporting task + final ReportingTaskNode reportingTask = locateReportingTask(reportingTaskDTO.getId()); + + // ensure we can perform the update + verifyUpdate(reportingTask, reportingTaskDTO); + + // perform the update + configureReportingTask(reportingTask, reportingTaskDTO); + + // configure scheduled state + // see if an update is necessary + if (isNotNull(reportingTaskDTO.getState())) { + final ScheduledState purposedScheduledState = ScheduledState.valueOf(reportingTaskDTO.getState()); + + // only attempt an action if it is changing + if (!purposedScheduledState.equals(reportingTask.getScheduledState())) { + try { + // perform the appropriate action + switch (purposedScheduledState) { + case RUNNING: + reportingTaskProvider.startReportingTask(reportingTask); + break; + case STOPPED: + switch (reportingTask.getScheduledState()) { + case RUNNING: + reportingTaskProvider.stopReportingTask(reportingTask); + break; + case DISABLED: + reportingTaskProvider.enableReportingTask(reportingTask); + break; + } + break; + case DISABLED: + reportingTaskProvider.disableReportingTask(reportingTask); + break; + } + } catch (IllegalStateException | ProcessorLifeCycleException ise) { + throw new NiFiCoreException(ise.getMessage(), ise); + } catch (RejectedExecutionException ree) { + throw new NiFiCoreException("Unable to schedule all tasks for the specified reporting task.", ree); + } catch (NullPointerException npe) { + throw new NiFiCoreException("Unable to update reporting task run state.", npe); + } catch (Exception e) { + throw new NiFiCoreException("Unable to update reporting task run state: " + e, e); + } + } + } + + return reportingTask; + } + + /** + * Validates the specified configuration for the specified reporting task. + * + * @param reportingTask + * @param reportingTaskDTO + * @return + */ + private List validateProposedConfiguration(final ReportingTaskNode reportingTask, final ReportingTaskDTO reportingTaskDTO) { + final List validationErrors = new ArrayList<>(); + + // get the current scheduling strategy + SchedulingStrategy schedulingStrategy = reportingTask.getSchedulingStrategy(); + + // validate the new scheduling strategy if appropriate + if (isNotNull(reportingTaskDTO.getSchedulingStrategy())) { + try { + // this will be the new scheduling strategy so use it + schedulingStrategy = SchedulingStrategy.valueOf(reportingTaskDTO.getSchedulingStrategy()); + } catch (IllegalArgumentException iae) { + validationErrors.add(String.format("Scheduling strategy: Value must be one of [%s]", StringUtils.join(SchedulingStrategy.values(), ", "))); + } + } + + // validate the scheduling period based on the scheduling strategy + if (isNotNull(reportingTaskDTO.getSchedulingPeriod())) { + switch (schedulingStrategy) { + case TIMER_DRIVEN: + final Matcher schedulingMatcher = FormatUtils.TIME_DURATION_PATTERN.matcher(reportingTaskDTO.getSchedulingPeriod()); + if (!schedulingMatcher.matches()) { + validationErrors.add("Scheduling period is not a valid time duration (ie 30 sec, 5 min)"); + } + break; + case CRON_DRIVEN: + try { + new CronExpression(reportingTaskDTO.getSchedulingPeriod()); + } catch (final ParseException pe) { + throw new IllegalArgumentException(String.format("Scheduling Period '%s' is not a valid cron expression: %s", reportingTaskDTO.getSchedulingPeriod(), pe.getMessage())); + } catch (final Exception e) { + throw new IllegalArgumentException("Scheduling Period is not a valid cron expression: " + reportingTaskDTO.getSchedulingPeriod()); + } + break; + } + } + + return validationErrors; + } + + @Override + public void verifyDelete(final String reportingTaskId) { + final ReportingTaskNode reportingTask = locateReportingTask(reportingTaskId); + reportingTask.verifyCanDelete(); + } + + @Override + public void verifyUpdate(final ReportingTaskDTO reportingTaskDTO) { + final ReportingTaskNode reportingTask = locateReportingTask(reportingTaskDTO.getId()); + verifyUpdate(reportingTask, reportingTaskDTO); + } + + /** + * Verifies the reporting task can be updated. + * + * @param reportingTask + * @param reportingTaskDTO + */ + private void verifyUpdate(final ReportingTaskNode reportingTask, final ReportingTaskDTO reportingTaskDTO) { + // ensure the state, if specified, is valid + if (isNotNull(reportingTaskDTO.getState())) { + try { + final ScheduledState purposedScheduledState = ScheduledState.valueOf(reportingTaskDTO.getState()); + + // only attempt an action if it is changing + if (!purposedScheduledState.equals(reportingTask.getScheduledState())) { + // perform the appropriate action + switch (purposedScheduledState) { + case RUNNING: + reportingTask.verifyCanStart(); + break; + case STOPPED: + switch (reportingTask.getScheduledState()) { + case RUNNING: + reportingTask.verifyCanStop(); + break; + case DISABLED: + reportingTask.verifyCanEnable(); + break; + } + break; + case DISABLED: + reportingTask.verifyCanDisable(); + break; + } + } + } catch (IllegalArgumentException iae) { + throw new IllegalArgumentException(String.format( + "The specified reporting task state (%s) is not valid. Valid options are 'RUNNING', 'STOPPED', and 'DISABLED'.", + reportingTaskDTO.getState())); + } + } + + boolean modificationRequest = false; + if (isAnyNotNull(reportingTaskDTO.getName(), + reportingTaskDTO.getSchedulingStrategy(), + reportingTaskDTO.getSchedulingPeriod(), + reportingTaskDTO.getAnnotationData(), + reportingTaskDTO.getProperties())) { + modificationRequest = true; + + // validate the request + final List requestValidation = validateProposedConfiguration(reportingTask, reportingTaskDTO); + + // ensure there was no validation errors + if (!requestValidation.isEmpty()) { + throw new ValidationException(requestValidation); + } + } + + if (modificationRequest) { + reportingTask.verifyCanUpdate(); + } + } + + /** + * Configures the specified reporting task. + * + * @param reportingTask + * @param reportingTaskDTO + */ + private void configureReportingTask(final ReportingTaskNode reportingTask, final ReportingTaskDTO reportingTaskDTO) { + final String name = reportingTaskDTO.getName(); + final String schedulingStrategy = reportingTaskDTO.getSchedulingStrategy(); + final String schedulingPeriod = reportingTaskDTO.getSchedulingPeriod(); + final String annotationData = reportingTaskDTO.getAnnotationData(); + final String comments = reportingTaskDTO.getComments(); + final Map properties = reportingTaskDTO.getProperties(); + + // ensure scheduling strategy is set first + if (isNotNull(schedulingStrategy)) { + reportingTask.setSchedulingStrategy(SchedulingStrategy.valueOf(schedulingStrategy)); + } + + if (isNotNull(name)) { + reportingTask.setName(name); + } + if (isNotNull(schedulingPeriod)) { + reportingTask.setScheduldingPeriod(schedulingPeriod); + } + if (isNotNull(annotationData)) { + reportingTask.setAnnotationData(annotationData); + } + if (isNotNull(comments)) { + reportingTask.setComments(comments); + } + if (isNotNull(properties)) { + for (final Map.Entry entry : properties.entrySet()) { + final String propName = entry.getKey(); + final String propVal = entry.getValue(); + if (isNotNull(propName) && propVal == null) { + reportingTask.removeProperty(propName); + } else if (isNotNull(propName)) { + reportingTask.setProperty(propName, propVal); + } + } + } + } + + /** + * Deletes the specified reporting task. + * + * @param reportingTaskId The reporting task id + */ + @Override + public void deleteReportingTask(String reportingTaskId) { + final ReportingTaskNode reportingTask = locateReportingTask(reportingTaskId); + reportingTaskProvider.removeReportingTask(reportingTask); + } + + /* setters */ + public void setReportingTaskProvider(ReportingTaskProvider reportingTaskProvider) { + this.reportingTaskProvider = reportingTaskProvider; + } +} diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardSnippetDAO.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardSnippetDAO.java index 92e3a8d90a..6447464a84 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardSnippetDAO.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardSnippetDAO.java @@ -26,9 +26,11 @@ import org.apache.nifi.controller.ProcessorNode; import org.apache.nifi.controller.Snippet; import org.apache.nifi.controller.StandardSnippet; import org.apache.nifi.controller.exception.ProcessorInstantiationException; +import org.apache.nifi.controller.service.ControllerServiceNode; import org.apache.nifi.groups.ProcessGroup; import org.apache.nifi.web.NiFiCoreException; import org.apache.nifi.web.ResourceNotFoundException; +import org.apache.nifi.web.api.dto.ControllerServiceDTO; import org.apache.nifi.web.api.dto.FlowSnippetDTO; import org.apache.nifi.web.api.dto.ProcessGroupDTO; import org.apache.nifi.web.api.dto.ProcessorConfigDTO; @@ -36,7 +38,6 @@ import org.apache.nifi.web.api.dto.ProcessorDTO; import org.apache.nifi.web.api.dto.SnippetDTO; import org.apache.nifi.web.dao.SnippetDAO; import org.apache.nifi.web.util.SnippetUtils; - import org.apache.commons.lang3.StringUtils; /** @@ -285,9 +286,13 @@ public class StandardSnippetDAO implements SnippetDAO { if (snippet != null) { // go through each processor if specified if (snippet.getProcessors() != null) { - lookupSensitiveProperties(snippet.getProcessors()); + lookupSensitiveProcessorProperties(snippet.getProcessors()); } + if ( snippet.getControllerServices() != null ) { + lookupSensitiveControllerServiceProperties(snippet.getControllerServices()); + } + // go through each process group if specified if (snippet.getProcessGroups() != null) { for (final ProcessGroupDTO group : snippet.getProcessGroups()) { @@ -303,7 +308,7 @@ public class StandardSnippetDAO implements SnippetDAO { * * @param snippet */ - private void lookupSensitiveProperties(final Set processors) { + private void lookupSensitiveProcessorProperties(final Set processors) { final ProcessGroup rootGroup = flowController.getGroup(flowController.getRootGroupId()); // go through each processor @@ -331,6 +336,31 @@ public class StandardSnippetDAO implements SnippetDAO { } } } + + private void lookupSensitiveControllerServiceProperties(final Set controllerServices) { + // go through each service + for (final ControllerServiceDTO serviceDTO : controllerServices) { + + // ensure that some property configuration have been specified + final Map serviceProperties = serviceDTO.getProperties(); + if (serviceProperties != null) { + // find the corresponding controller service + final ControllerServiceNode serviceNode = flowController.getControllerServiceNode(serviceDTO.getId()); + if (serviceNode == null) { + throw new IllegalArgumentException(String.format("Unable to create snippet because Controller Service '%s' could not be found", serviceDTO.getId())); + } + + // look for sensitive properties get the actual value + for (Entry entry : serviceNode.getProperties().entrySet()) { + final PropertyDescriptor descriptor = entry.getKey(); + + if (descriptor.isSensitive()) { + serviceProperties.put(descriptor.getName(), entry.getValue()); + } + } + } + } + } /* setters */ public void setFlowController(FlowController flowController) { diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/spring/ControllerServiceProviderFactoryBean.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/spring/ControllerServiceProviderFactoryBean.java new file mode 100644 index 0000000000..5c10de6af5 --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/spring/ControllerServiceProviderFactoryBean.java @@ -0,0 +1,68 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.web.spring; + +import org.apache.nifi.cluster.manager.impl.WebClusterManager; +import org.apache.nifi.controller.FlowController; +import org.apache.nifi.controller.service.ControllerServiceProvider; +import org.apache.nifi.util.NiFiProperties; +import org.springframework.beans.BeansException; +import org.springframework.beans.factory.FactoryBean; +import org.springframework.context.ApplicationContext; +import org.springframework.context.ApplicationContextAware; + +/** + * + */ +public class ControllerServiceProviderFactoryBean implements FactoryBean, ApplicationContextAware { + + private ApplicationContext context; + private ControllerServiceProvider controllerServiceProvider; + private NiFiProperties properties; + + @Override + public Object getObject() throws Exception { + if (controllerServiceProvider == null) { + if (properties.isClusterManager()) { + controllerServiceProvider = context.getBean("clusterManager", WebClusterManager.class); + } else { + controllerServiceProvider = context.getBean("flowController", FlowController.class); + } + } + + return controllerServiceProvider; + } + + @Override + public Class getObjectType() { + return ControllerServiceProvider.class; + } + + @Override + public boolean isSingleton() { + return true; + } + + public void setProperties(NiFiProperties properties) { + this.properties = properties; + } + + @Override + public void setApplicationContext(ApplicationContext context) throws BeansException { + this.context = context; + } +} diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/spring/OptimisticLockingManagerFactoryBean.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/spring/OptimisticLockingManagerFactoryBean.java new file mode 100644 index 0000000000..84367932b5 --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/spring/OptimisticLockingManagerFactoryBean.java @@ -0,0 +1,67 @@ +/* + * 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.web.spring; + +import org.apache.nifi.util.NiFiProperties; +import org.apache.nifi.web.OptimisticLockingManager; +import org.apache.nifi.web.StandardOptimisticLockingManager; +import org.springframework.beans.BeansException; +import org.springframework.beans.factory.FactoryBean; +import org.springframework.context.ApplicationContext; +import org.springframework.context.ApplicationContextAware; + +/** + * + */ +public class OptimisticLockingManagerFactoryBean implements FactoryBean, ApplicationContextAware { + + private ApplicationContext context; + private OptimisticLockingManager optimisticLockingManager; + private NiFiProperties properties; + + @Override + public Object getObject() throws Exception { + if (optimisticLockingManager == null) { + if (properties.isClusterManager()) { + optimisticLockingManager = context.getBean("clusterManagerOptimisticLockingManager", OptimisticLockingManager.class); + } else { + optimisticLockingManager = new StandardOptimisticLockingManager(); + } + } + + return optimisticLockingManager; + } + + @Override + public Class getObjectType() { + return OptimisticLockingManager.class; + } + + @Override + public boolean isSingleton() { + return true; + } + + public void setProperties(NiFiProperties properties) { + this.properties = properties; + } + + @Override + public void setApplicationContext(ApplicationContext context) throws BeansException { + this.context = context; + } +} diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/spring/ReportingTaskProviderFactoryBean.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/spring/ReportingTaskProviderFactoryBean.java new file mode 100644 index 0000000000..d344fa6a06 --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/spring/ReportingTaskProviderFactoryBean.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.web.spring; + +import org.apache.nifi.cluster.manager.impl.WebClusterManager; +import org.apache.nifi.controller.FlowController; +import org.apache.nifi.controller.reporting.ReportingTaskProvider; +import org.apache.nifi.util.NiFiProperties; +import org.apache.nifi.web.dao.ControllerServiceDAO; +import org.springframework.beans.BeansException; +import org.springframework.beans.factory.FactoryBean; +import org.springframework.context.ApplicationContext; +import org.springframework.context.ApplicationContextAware; + +/** + * + */ +public class ReportingTaskProviderFactoryBean implements FactoryBean, ApplicationContextAware { + + private ApplicationContext context; + private ReportingTaskProvider reportingTaskProvider; + private NiFiProperties properties; + + @Override + public Object getObject() throws Exception { + if (reportingTaskProvider == null) { + if (properties.isClusterManager()) { + reportingTaskProvider = context.getBean("clusterManager", WebClusterManager.class); + } else { + reportingTaskProvider = context.getBean("flowController", FlowController.class); + } + } + + return reportingTaskProvider; + } + + @Override + public Class getObjectType() { + return ReportingTaskProvider.class; + } + + @Override + public boolean isSingleton() { + return true; + } + + public void setProperties(NiFiProperties properties) { + this.properties = properties; + } + + @Override + public void setApplicationContext(ApplicationContext context) throws BeansException { + this.context = context; + } +} diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/util/Availability.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/util/Availability.java new file mode 100644 index 0000000000..29ba4f8e21 --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/util/Availability.java @@ -0,0 +1,34 @@ +/* + * 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.web.util; + +/** + * Where a given controller service or reporting task should run. + */ +public enum Availability { + + /** + * Service or reporting task will run only on the NiFi Cluster Manager (NCM) + */ + NCM, + + /** + * Service or reporting task will run only on NiFi Nodes (or standalone + * instance, if not clustered) + */ + NODE; +} diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/util/SnippetUtils.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/util/SnippetUtils.java index 86530947cc..fa9bc416eb 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/util/SnippetUtils.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/util/SnippetUtils.java @@ -18,6 +18,7 @@ package org.apache.nifi.web.util; import java.nio.charset.StandardCharsets; import java.util.ArrayList; +import java.util.Collection; import java.util.HashMap; import java.util.HashSet; import java.util.LinkedHashSet; @@ -28,6 +29,7 @@ import java.util.UUID; import org.apache.nifi.cluster.context.ClusterContext; import org.apache.nifi.cluster.context.ClusterContextThreadLocal; +import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.connectable.ConnectableType; import org.apache.nifi.connectable.Connection; import org.apache.nifi.connectable.Funnel; @@ -37,17 +39,22 @@ import org.apache.nifi.controller.ProcessorNode; import org.apache.nifi.controller.ScheduledState; import org.apache.nifi.controller.Snippet; import org.apache.nifi.controller.label.Label; +import org.apache.nifi.controller.service.ControllerServiceNode; +import org.apache.nifi.controller.service.ControllerServiceState; import org.apache.nifi.groups.ProcessGroup; import org.apache.nifi.groups.RemoteProcessGroup; import org.apache.nifi.web.api.dto.ConnectableDTO; import org.apache.nifi.web.api.dto.ConnectionDTO; +import org.apache.nifi.web.api.dto.ControllerServiceDTO; import org.apache.nifi.web.api.dto.DtoFactory; import org.apache.nifi.web.api.dto.FlowSnippetDTO; import org.apache.nifi.web.api.dto.FunnelDTO; import org.apache.nifi.web.api.dto.LabelDTO; import org.apache.nifi.web.api.dto.PortDTO; import org.apache.nifi.web.api.dto.ProcessGroupDTO; +import org.apache.nifi.web.api.dto.ProcessorConfigDTO; import org.apache.nifi.web.api.dto.ProcessorDTO; +import org.apache.nifi.web.api.dto.PropertyDescriptorDTO; import org.apache.nifi.web.api.dto.RemoteProcessGroupContentsDTO; import org.apache.nifi.web.api.dto.RemoteProcessGroupDTO; import org.apache.nifi.web.api.dto.RemoteProcessGroupPortDTO; @@ -181,11 +188,100 @@ public final class SnippetUtils { snippetDto.setRemoteProcessGroups(remoteProcessGroups); } + addControllerServicesToSnippet(snippetDto); + return snippetDto; } + + private void addControllerServicesToSnippet(final FlowSnippetDTO snippetDto) { + for ( final ProcessorDTO processorDto : snippetDto.getProcessors() ) { + addControllerServicesToSnippet(snippetDto, processorDto); + } + + for ( final ProcessGroupDTO processGroupDto : snippetDto.getProcessGroups() ) { + final FlowSnippetDTO childGroupDto = processGroupDto.getContents(); + addControllerServicesToSnippet(childGroupDto); + } + } + + private void addControllerServicesToSnippet(final FlowSnippetDTO snippet, final ProcessorDTO processorDto) { + final ProcessorConfigDTO configDto = processorDto.getConfig(); + if ( configDto == null ) { + return; + } + + final Map descriptors = configDto.getDescriptors(); + final Map properties = configDto.getProperties(); + + if ( properties != null && descriptors != null ) { + for ( final Map.Entry entry : properties.entrySet() ) { + final String propName = entry.getKey(); + final String propValue = entry.getValue(); + if ( propValue == null ) { + continue; + } + + final PropertyDescriptorDTO propertyDescriptorDto = descriptors.get(propName); + if ( propertyDescriptorDto != null && propertyDescriptorDto.isIdentifiesControllerService() ) { + final ControllerServiceNode serviceNode = flowController.getControllerServiceNode(propValue); + if ( serviceNode != null ) { + addControllerServicesToSnippet(snippet, serviceNode); + } + } + } + } + } + + private void addControllerServicesToSnippet(final FlowSnippetDTO snippet, final ControllerServiceNode serviceNode) { + if ( isServicePresent(serviceNode.getIdentifier(), snippet.getControllerServices()) ) { + return; + } + + final ControllerServiceDTO serviceNodeDto = dtoFactory.createControllerServiceDto(serviceNode); + Set existingServiceDtos = snippet.getControllerServices(); + if ( existingServiceDtos == null ) { + existingServiceDtos = new HashSet<>(); + snippet.setControllerServices(existingServiceDtos); + } + existingServiceDtos.add(serviceNodeDto); + for ( final Map.Entry entry : serviceNode.getProperties().entrySet() ) { + final PropertyDescriptor descriptor = entry.getKey(); + final String propertyValue = entry.getValue(); + + if ( descriptor.getControllerServiceDefinition() != null ) { + final ControllerServiceNode referencedNode = flowController.getControllerServiceNode(propertyValue); + if ( referencedNode == null ) { + throw new IllegalStateException("Controller Service with ID " + propertyValue + " is referenced in template but cannot be found"); + } + + final String referencedNodeId = referencedNode.getIdentifier(); + + final boolean alreadyPresent = isServicePresent(referencedNodeId, snippet.getControllerServices()); + if ( !alreadyPresent ) { + addControllerServicesToSnippet(snippet, referencedNode); + } + } + } + } + + private boolean isServicePresent(final String serviceId, final Collection services) { + if ( services == null ) { + return false; + } + + for ( final ControllerServiceDTO existingService : services ) { + if ( serviceId.equals(existingService.getId()) ) { + return true; + } + } + + return false; + } + + public FlowSnippetDTO copy(final FlowSnippetDTO snippetContents, final ProcessGroup group) { - final FlowSnippetDTO snippetCopy = copyContentsForGroup(snippetContents, group.getIdentifier(), null); + final FlowSnippetDTO snippetCopy = copyContentsForGroup(snippetContents, group.getIdentifier(), null, null); resolveNameConflicts(snippetCopy, group); return snippetCopy; } @@ -240,9 +336,48 @@ public final class SnippetUtils { } } - private FlowSnippetDTO copyContentsForGroup(final FlowSnippetDTO snippetContents, final String groupId, final Map parentConnectableMap) { + private FlowSnippetDTO copyContentsForGroup(final FlowSnippetDTO snippetContents, final String groupId, final Map parentConnectableMap, Map serviceIdMap) { final FlowSnippetDTO snippetContentsCopy = new FlowSnippetDTO(); + // + // Copy the Controller Services + // + if ( serviceIdMap == null ) { + serviceIdMap = new HashMap<>(); + final Set services = new HashSet<>(); + if ( snippetContents.getControllerServices() != null ) { + for (final ControllerServiceDTO serviceDTO : snippetContents.getControllerServices() ) { + final ControllerServiceDTO service = dtoFactory.copy(serviceDTO); + service.setId(generateId(serviceDTO.getId())); + service.setState(ControllerServiceState.DISABLED.name()); + services.add(service); + + // Map old service ID to new service ID so that we can make sure that we reference the new ones. + serviceIdMap.put(serviceDTO.getId(), service.getId()); + } + } + + // if there is any controller service that maps to another controller service, update the id's + for ( final ControllerServiceDTO serviceDTO : services ) { + final Map properties = serviceDTO.getProperties(); + final Map descriptors = serviceDTO.getDescriptors(); + if ( properties != null && descriptors != null ) { + for ( final PropertyDescriptorDTO descriptor : descriptors.values() ) { + if ( descriptor.isIdentifiesControllerService() ) { + final String currentServiceId = properties.get(descriptor.getName()); + if ( currentServiceId == null ) { + continue; + } + + final String newServiceId = serviceIdMap.get(currentServiceId); + properties.put(descriptor.getName(), newServiceId); + } + } + } + } + snippetContentsCopy.setControllerServices(services); + } + // // Copy the labels // @@ -332,6 +467,9 @@ public final class SnippetUtils { } snippetContentsCopy.setProcessors(processors); + // if there is any controller service that maps to another controller service, update the id's + updateControllerServiceIdentifiers(snippetContentsCopy, serviceIdMap); + // // Copy ProcessGroups // @@ -344,7 +482,7 @@ public final class SnippetUtils { cp.setParentGroupId(groupId); // copy the contents of this group - we do not copy via the dto factory since we want to specify new ids - final FlowSnippetDTO contentsCopy = copyContentsForGroup(groupDTO.getContents(), cp.getId(), connectableMap); + final FlowSnippetDTO contentsCopy = copyContentsForGroup(groupDTO.getContents(), cp.getId(), connectableMap, serviceIdMap); cp.setContents(contentsCopy); groups.add(cp); } @@ -396,6 +534,43 @@ public final class SnippetUtils { return snippetContentsCopy; } + + + private void updateControllerServiceIdentifiers(final FlowSnippetDTO snippet, final Map serviceIdMap) { + final Set processors = snippet.getProcessors(); + if ( processors != null ) { + for ( final ProcessorDTO processor : processors ) { + updateControllerServiceIdentifiers(processor.getConfig(), serviceIdMap); + } + } + + for ( final ProcessGroupDTO processGroupDto : snippet.getProcessGroups() ) { + updateControllerServiceIdentifiers(processGroupDto.getContents(), serviceIdMap); + } + } + + private void updateControllerServiceIdentifiers(final ProcessorConfigDTO configDto, final Map serviceIdMap) { + if ( configDto == null ) { + return; + } + + final Map properties = configDto.getProperties(); + final Map descriptors = configDto.getDescriptors(); + if ( properties != null && descriptors != null ) { + for ( final PropertyDescriptorDTO descriptor : descriptors.values() ) { + if ( descriptor.isIdentifiesControllerService() ) { + final String currentServiceId = properties.get(descriptor.getName()); + if ( currentServiceId == null ) { + continue; + } + + final String newServiceId = serviceIdMap.get(currentServiceId); + properties.put(descriptor.getName(), newServiceId); + } + } + } + } + /** * Generates a new id for the current id that is specified. If no seed is diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/resources/nifi-web-api-context.xml b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/resources/nifi-web-api-context.xml index a822442bca..bf4f245ace 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/resources/nifi-web-api-context.xml +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/resources/nifi-web-api-context.xml @@ -24,11 +24,17 @@ http://www.springframework.org/schema/context http://www.springframework.org/schema/context/spring-context-3.1.xsd http://www.springframework.org/schema/aop http://www.springframework.org/schema/aop/spring-aop-3.1.xsd"> + + + + + + + + - - - - + + @@ -40,8 +46,7 @@ - - + @@ -75,6 +80,12 @@ + + + + + + @@ -101,22 +112,35 @@ + + - + + - + + + + + + + + + + + @@ -133,6 +157,16 @@ + + + + + + + + + + @@ -303,6 +337,16 @@ + + + + + + + + + + diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/integration/util/NiFiTestServer.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/integration/util/NiFiTestServer.java index dd9bb73276..1f286096bb 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/integration/util/NiFiTestServer.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/integration/util/NiFiTestServer.java @@ -18,12 +18,14 @@ package org.apache.nifi.integration.util; import com.sun.jersey.api.client.Client; import java.io.File; +import java.util.Collections; import javax.servlet.ServletContext; import org.apache.nifi.util.NiFiProperties; import org.apache.nifi.web.util.WebUtils; import org.apache.commons.lang3.StringUtils; import org.apache.nifi.framework.security.util.SslContextFactory; import org.apache.nifi.services.FlowService; +import org.apache.nifi.ui.extension.UiExtensionMapping; import org.eclipse.jetty.server.HttpConfiguration; import org.eclipse.jetty.server.HttpConnectionFactory; import org.eclipse.jetty.server.SecureRequestCustomizer; @@ -160,6 +162,9 @@ public class NiFiTestServer { */ public void startServer() throws Exception { jetty.start(); + + // ensure the ui extensions are set + webappContext.getServletContext().setAttribute("nifi-ui-extensions", new UiExtensionMapping(Collections.EMPTY_MAP)); } /** 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 e9781f8543..a9f0c3e2fc 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 @@ -27,5 +27,17 @@ org.apache.nifi nifi-api
      + + org.apache.nifi + nifi-administration + + + org.apache.nifi + nifi-web-security + + + org.apache.nifi + nifi-framework-cluster-web + diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-optimistic-locking/src/main/java/org/apache/nifi/web/ConfigurationRequest.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-optimistic-locking/src/main/java/org/apache/nifi/web/ConfigurationRequest.java new file mode 100644 index 0000000000..939c3f08fb --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-optimistic-locking/src/main/java/org/apache/nifi/web/ConfigurationRequest.java @@ -0,0 +1,34 @@ +/* + * 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.web; + +/** + * Represents a request to configure. The implementations execute method will + * perform the configuration action. It will return type T which will be + * encapsulated in a ConfigurationSnapshot. + * + * @param + */ +public interface ConfigurationRequest { + + /** + * Executes a configuration action and returns the updated resulting configuration. + * + * @return The resulting configuration + */ + T execute(); +} diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-optimistic-locking/src/main/java/org/apache/nifi/web/ConfigurationSnapshot.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-optimistic-locking/src/main/java/org/apache/nifi/web/ConfigurationSnapshot.java index 6ad683cac7..8817d69c19 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-optimistic-locking/src/main/java/org/apache/nifi/web/ConfigurationSnapshot.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-optimistic-locking/src/main/java/org/apache/nifi/web/ConfigurationSnapshot.java @@ -22,36 +22,36 @@ package org.apache.nifi.web; */ public class ConfigurationSnapshot { - private Long revision; + private Long version; private T configuration; /** * Creates a new ConfigurationSnapshot. * - * @param revision The model revision + * @param version The revision version */ - public ConfigurationSnapshot(Long revision) { - this(revision, null); + public ConfigurationSnapshot(Long version) { + this(version, null); } /** * Creates a new ConfigurationSnapshot. * - * @param revision The model revision + * @param version The revision version * @param configuration The configuration */ - public ConfigurationSnapshot(Long revision, T configuration) { - this.revision = revision; + public ConfigurationSnapshot(Long version, T configuration) { + this.version = version; this.configuration = configuration; } /** - * Get the new model revision. + * Get the revision version. * - * @return The model revision + * @return The revision version */ - public Long getRevision() { - return revision; + public Long getVersion() { + return version; } /** diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-optimistic-locking/src/main/java/org/apache/nifi/web/FlowModification.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-optimistic-locking/src/main/java/org/apache/nifi/web/FlowModification.java new file mode 100644 index 0000000000..f6bccb1b4f --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-optimistic-locking/src/main/java/org/apache/nifi/web/FlowModification.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.web; + +/** + * Records a flow modification. This includes the resulting revision and the + * user that performed the modification. + */ +public class FlowModification { + + private final Revision revision; + private final String lastModifier; + + /** + * Creates a new FlowModification. + * + * @param revision + * @param lastModifier + */ + public FlowModification(Revision revision, String lastModifier) { + this.revision = revision; + this.lastModifier = lastModifier; + } + + /** + * Get the revision. + * + * @return + */ + public Revision getRevision() { + return revision; + } + + /** + * Get the last modifier. + * + * @return + */ + public String getLastModifier() { + return lastModifier; + } + +} diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-optimistic-locking/src/main/java/org/apache/nifi/web/OptimisticLockingManager.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-optimistic-locking/src/main/java/org/apache/nifi/web/OptimisticLockingManager.java index b045247e97..4c54b7cb0d 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-optimistic-locking/src/main/java/org/apache/nifi/web/OptimisticLockingManager.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-optimistic-locking/src/main/java/org/apache/nifi/web/OptimisticLockingManager.java @@ -26,70 +26,28 @@ package org.apache.nifi.web; public interface OptimisticLockingManager { /** - * Checks the specified revision against the current revision. If the check - * succeeds, then the current revision's version is incremented and the - * current revision's client ID is set to the given revision's client ID. - * - * If the given revision's version is null, then the revision's client ID - * must match for the current revision's client ID for the check to succeed. - * - * If the versions and the clientIds do not match, then an - * InvalidRevisionException. - * - * @param revision the revision to check - * - * @return the current revision - * - * @throws InvalidRevisionException if the given revision does not match the - * current revision + * Attempts to execute the specified configuration request using the specified revision within a lock. + * + * @param + * @param revision + * @param configurationRequest + * @return */ - Revision checkRevision(Revision revision) throws InvalidRevisionException; + ConfigurationSnapshot configureFlow(Revision revision, ConfigurationRequest configurationRequest); + + /** + * Updates the revision using the specified revision within a lock. + * + * @param updateRevision + */ + void setRevision(UpdateRevision updateRevision); /** - * Returns true if the given revision matches the current revision. - * - * @param revision a revision - * @return true if given revision is current; false otherwise. + * Returns the last flow modification. This is a combination of the revision and the user + * who performed the modification. + * + * @return the last modification */ - boolean isCurrent(Revision revision); + FlowModification getLastModification(); - /** - * @return the current revision - */ - Revision getRevision(); - - /** - * Sets the current revision. - * - * @param revision a revision - */ - void setRevision(Revision revision); - - /** - * Increments the current revision's version. - * - * @return the current revision - */ - Revision incrementRevision(); - - /** - * Increments the current revision's version and sets the current revision's - * client ID to the given client ID. - * - * @param clientId a client ID - * @return the current revision - */ - Revision incrementRevision(String clientId); - - /** - * @return the last modifier. - */ - String getLastModifier(); - - /** - * Sets the last modifier. - * - * @param lastModifier the last modifier - */ - void setLastModifier(String lastModifier); } diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-optimistic-locking/src/main/java/org/apache/nifi/web/StandardOptimisticLockingManager.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-optimistic-locking/src/main/java/org/apache/nifi/web/StandardOptimisticLockingManager.java index 8da6d2339c..23ef8ebf60 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-optimistic-locking/src/main/java/org/apache/nifi/web/StandardOptimisticLockingManager.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-optimistic-locking/src/main/java/org/apache/nifi/web/StandardOptimisticLockingManager.java @@ -16,6 +16,14 @@ */ package org.apache.nifi.web; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; +import org.apache.nifi.cluster.context.ClusterContext; +import org.apache.nifi.cluster.context.ClusterContextThreadLocal; +import org.apache.nifi.web.security.user.NiFiUserUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + /** * Implements the OptimisticLockingManager interface. * @@ -23,55 +31,129 @@ package org.apache.nifi.web; */ public class StandardOptimisticLockingManager implements OptimisticLockingManager { + private static final Logger logger = LoggerFactory.getLogger(StandardOptimisticLockingManager.class); + + private static final String INVALID_REVISION_ERROR = "Given revision %s does not match current revision %s."; + private static final String SYNC_ERROR = "This NiFi instance has been updated by '%s'. Please refresh to synchronize the view."; + private Revision currentRevision = new Revision(0L, ""); - private String lastModifier = "unknown"; + private final Lock lock = new ReentrantLock(); + + private void lock() { + lock.lock(); + } + + private void unlock() { + lock.unlock(); + } + + private void checkRevision(final Revision revision) { + final FlowModification lastMod = getLastModification(); + + // with lock, verify revision + boolean approved = lastMod.getRevision().equals(revision); + + if (!approved) { + logger.debug("Revision check failed because current revision is " + lastMod.getRevision() + " but supplied revision is " + revision); + + if (lastMod.getRevision().getClientId() == null || lastMod.getRevision().getClientId().trim().isEmpty() || lastMod.getRevision().getVersion() == null) { + throw new InvalidRevisionException(String.format(INVALID_REVISION_ERROR, revision, lastMod.getRevision())); + } else { + throw new InvalidRevisionException(String.format(SYNC_ERROR, lastMod.getLastModifier())); + } + } + } + + private Revision updateRevision(final Revision updatedRevision) { + // record the current modification + setLastModification(new FlowModification(updatedRevision, NiFiUserUtils.getNiFiUserName())); + + // return the revision + return updatedRevision; + } @Override - public Revision checkRevision(Revision revision) throws InvalidRevisionException { - if (currentRevision.equals(revision) == false) { - throw new InvalidRevisionException(String.format("Given revision %s does not match current revision %s.", revision, currentRevision)); - } else { - currentRevision = revision.increment(revision.getClientId()); - return currentRevision; + public ConfigurationSnapshot configureFlow(Revision revision, ConfigurationRequest configurationRequest) { + lock(); + try { + // check the revision + checkRevision(revision); + + // execute the configuration request + final T result = configurationRequest.execute(); + + // update the revision + final Revision newRevision = updateRevision(incrementRevision(revision.getClientId())); + + // build the result + return new ConfigurationSnapshot(newRevision.getVersion(), result); + } finally { + unlock(); } } @Override - public boolean isCurrent(Revision revision) { - return currentRevision.equals(revision); + public void setRevision(UpdateRevision updateRevision) { + lock(); + try { + final Revision updatedRevision = updateRevision.execute(getLastModification().getRevision()); + + // update the revision + if (updatedRevision != null) { + updateRevision(updatedRevision); + } + } finally { + unlock(); + } } - + @Override - public Revision getRevision() { - return currentRevision; + public FlowModification getLastModification() { + lock(); + try { + final Revision revision; + final ClusterContext ctx = ClusterContextThreadLocal.getContext(); + if (ctx == null || ctx.getRevision() == null) { + revision = currentRevision; + } else { + revision = ctx.getRevision(); + } + + return new FlowModification(revision, lastModifier); + } finally { + unlock(); + } } - - @Override - public void setRevision(Revision revision) { - currentRevision = revision; + + private void setLastModification(final FlowModification lastModification) { + lock(); + try { + // record the last modifier + lastModifier = lastModification.getLastModifier(); + + // record the updated revision in the cluster context if possible + final ClusterContext ctx = ClusterContextThreadLocal.getContext(); + if (ctx != null) { + ctx.setRevision(lastModification.getRevision()); + } else { + currentRevision = lastModification.getRevision(); + } + } finally { + unlock(); + } } - - @Override - public Revision incrementRevision() { - currentRevision = currentRevision.increment(); - return currentRevision; + + private Revision incrementRevision(String clientId) { + final Revision current = getLastModification().getRevision(); + + final long incrementedVersion; + if (current.getVersion() == null) { + incrementedVersion = 0; + } else { + incrementedVersion = current.getVersion() + 1; + } + return new Revision(incrementedVersion, clientId); } - - @Override - public Revision incrementRevision(String clientId) { - currentRevision = currentRevision.increment(clientId); - return currentRevision; - } - - @Override - public String getLastModifier() { - return lastModifier; - } - - @Override - public void setLastModifier(String lastModifier) { - this.lastModifier = lastModifier; - } - + } diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-optimistic-locking/src/main/java/org/apache/nifi/web/UpdateRevision.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-optimistic-locking/src/main/java/org/apache/nifi/web/UpdateRevision.java new file mode 100644 index 0000000000..e691bbef0c --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-optimistic-locking/src/main/java/org/apache/nifi/web/UpdateRevision.java @@ -0,0 +1,31 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.web; + +/** + * Represents an action that will result in an updated revision. + */ +public interface UpdateRevision { + + /** + * Executes the action that will result in an updated revision + * + * @param currentRevision The current revision + * @return The updated revision + */ + Revision execute(Revision currentRevision); +} diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/user/NiFiUserUtils.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/user/NiFiUserUtils.java index 0209cf7055..acf37dc687 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/user/NiFiUserUtils.java +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/user/NiFiUserUtils.java @@ -79,4 +79,14 @@ public final class NiFiUserUtils { return user; } + + public static String getNiFiUserName() { + // get the nifi user to extract the username + NiFiUser user = NiFiUserUtils.getNiFiUser(); + if (user == null) { + return "unknown"; + } else { + return user.getUserName(); + } + } } 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 34ba05ea76..f2d4861876 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 @@ -265,12 +265,10 @@ ${staging.dir}/js/nf/canvas/nf-storage.js ${staging.dir}/js/nf/canvas/nf-snippet.js ${staging.dir}/js/nf/canvas/nf-canvas-toolbox.js - ${staging.dir}/js/nf/canvas/nf-custom-processor-ui.js + ${staging.dir}/js/nf/canvas/nf-custom-ui.js ${staging.dir}/js/nf/canvas/nf-registration.js - ${staging.dir}/js/nf/canvas/nf-processor-property-table.js - ${staging.dir}/js/nf/canvas/nf-processor-property-text-editor.js - ${staging.dir}/js/nf/canvas/nf-processor-property-nfel-editor.js - ${staging.dir}/js/nf/canvas/nf-processor-property-combo-editor.js + ${staging.dir}/js/nf/canvas/nf-controller-service.js + ${staging.dir}/js/nf/canvas/nf-reporting-task.js ${staging.dir}/js/nf/canvas/nf-processor-configuration.js ${staging.dir}/js/nf/nf-processor-details.js ${staging.dir}/js/nf/canvas/nf-process-group-configuration.js @@ -406,6 +404,8 @@ true ${project.build.directory}/${project.build.finalName}/css/nf-canvas-all.css + ${staging.dir}/css/reporting-task.css + ${staging.dir}/css/controller-service.css ${staging.dir}/css/processor-configuration.css ${staging.dir}/css/processor-details.css ${staging.dir}/css/process-group-configuration.css @@ -420,6 +420,8 @@ ${staging.dir}/css/registration.css ${staging.dir}/css/dialog.css ${staging.dir}/css/new-processor-dialog.css + ${staging.dir}/css/new-controller-service-dialog.css + ${staging.dir}/css/new-reporting-task-dialog.css ${staging.dir}/css/graph.css ${staging.dir}/css/header.css ${staging.dir}/css/main.css diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/resources/filters/canvas.properties b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/resources/filters/canvas.properties index 056a2af5a6..fd2bc170d4 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/resources/filters/canvas.properties +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/resources/filters/canvas.properties @@ -22,12 +22,10 @@ nf.canvas.script.tags=\n\ \n\ \n\ -\n\ +\n\ \n\ -\n\ -\n\ -\n\ -\n\ +\n\ +\n\ \n\ \n\ \n\ diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/pages/canvas.jsp b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/pages/canvas.jsp index ba6547576c..c81bb9d769 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/pages/canvas.jsp +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/pages/canvas.jsp @@ -29,6 +29,8 @@ + + @@ -45,6 +47,8 @@ + + @@ -76,13 +80,16 @@ + + + + - @@ -96,6 +103,8 @@ + + diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/pages/summary.jsp b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/pages/summary.jsp index fdfc806783..032509bd47 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/pages/summary.jsp +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/pages/summary.jsp @@ -25,6 +25,7 @@ ${nf.summary.style.tags} + @@ -40,6 +41,7 @@ + diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/partials/canvas/canvas-header.jsp b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/partials/canvas/canvas-header.jsp index 8eea811ef8..f312327d5a 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/partials/canvas/canvas-header.jsp +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/partials/canvas/canvas-header.jsp @@ -31,7 +31,7 @@ - + diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/partials/canvas/controller-service-configuration.jsp b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/partials/canvas/controller-service-configuration.jsp new file mode 100644 index 0000000000..533aa54175 --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/partials/canvas/controller-service-configuration.jsp @@ -0,0 +1,90 @@ +<%-- + 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. +--%> +<%@ page contentType="text/html" pageEncoding="UTF-8" session="false" %> +
      +
      +
      +
      +
      +
      +
      +
      Name
      +
      + +
      +
      + Enabled +
      +
      + +
      +
      +
      Id
      +
      + +
      +
      +
      +
      Type
      +
      + +
      +
      + +
      +
       
      +
      +
      +
      + Referencing Components + Info +
      +
      +
      +
      +
      +
      +
      +
      +
      +
      +
      + + +
      +
      +
      +
      +
      \ No newline at end of file diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/partials/canvas/disable-controller-service-dialog.jsp b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/partials/canvas/disable-controller-service-dialog.jsp new file mode 100644 index 0000000000..dc76282257 --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/partials/canvas/disable-controller-service-dialog.jsp @@ -0,0 +1,71 @@ +<%-- + 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. +--%> +<%@ page contentType="text/html" pageEncoding="UTF-8" session="false" %> +
      +
      +
      +
      +
      Service
      +
      + + +
      +
      +
      +
      Scope
      +
      + Service and referencing components + Info +
      +
      + +
      +
       
      +
      +
      +
      + Referencing Components + Info +
      +
      +
      +
      +
      +
      +
      +
      diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/partials/canvas/enable-controller-service-dialog.jsp b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/partials/canvas/enable-controller-service-dialog.jsp new file mode 100644 index 0000000000..14fe658538 --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/partials/canvas/enable-controller-service-dialog.jsp @@ -0,0 +1,70 @@ +<%-- + 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. +--%> +<%@ page contentType="text/html" pageEncoding="UTF-8" session="false" %> +
      +
      +
      +
      +
      Service
      +
      + + +
      +
      +
      +
      Scope
      +
      +
      +
      +
      + +
      +
       
      +
      +
      +
      + Referencing Components + Info +
      +
      +
      +
      +
      +
      +
      +
      diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/partials/canvas/new-controller-service-dialog.jsp b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/partials/canvas/new-controller-service-dialog.jsp new file mode 100644 index 0000000000..eb54ace556 --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/partials/canvas/new-controller-service-dialog.jsp @@ -0,0 +1,53 @@ +<%-- + 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. +--%> +<%@ page contentType="text/html" pageEncoding="UTF-8" session="false" %> +
      +
      +
      +
      + +
      +
      +
      + Displaying  of  +
      +
      +
      +
      +
      Tags
      +
      +
      +
      +
      +
      +
      +
      + +
      +
      + +
      +
      diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/partials/canvas/new-processor-dialog.jsp b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/partials/canvas/new-processor-dialog.jsp index 29b57c9627..df7766ca75 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/partials/canvas/new-processor-dialog.jsp +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/partials/canvas/new-processor-dialog.jsp @@ -26,13 +26,11 @@ Displaying  of  -
      +
      Tags
      -
        -
        -
          +
          diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/partials/canvas/new-processor-property-dialog.jsp b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/partials/canvas/new-processor-property-dialog.jsp deleted file mode 100644 index 9f58c2d956..0000000000 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/partials/canvas/new-processor-property-dialog.jsp +++ /dev/null @@ -1,34 +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. ---%> -<%@ page contentType="text/html" pageEncoding="UTF-8" session="false" %> -
          -
          -
          Property name
          -
          - -
          -
          Property value
          -
          -
          -
          -
          -
          -
          Ok
          -
          Cancel
          -
          -
          -
          diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/partials/canvas/new-reporting-task-dialog.jsp b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/partials/canvas/new-reporting-task-dialog.jsp new file mode 100644 index 0000000000..cfb39921c8 --- /dev/null +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/partials/canvas/new-reporting-task-dialog.jsp @@ -0,0 +1,53 @@ +<%-- + 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. +--%> +<%@ page contentType="text/html" pageEncoding="UTF-8" session="false" %> +
          +
          +
          +
          + +
          +
          +
          + Displaying  of  +
          +
          +
          +
          +
          Tags
          +
          +
          +
          +
          +
          +
          +
          + +
          +
          + +
          +
          diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/partials/canvas/processor-configuration.jsp b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/partials/canvas/processor-configuration.jsp index 9d9da2abd8..3516de7e88 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/partials/canvas/processor-configuration.jsp +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/partials/canvas/processor-configuration.jsp @@ -19,7 +19,7 @@
          -
          +
          Name
          @@ -71,7 +71,7 @@ Info
          -
          +
          @@ -90,7 +90,7 @@
          -
          +
          @@ -194,20 +194,13 @@
          -
          -
          -
          Required field
          -
          -
          -
          New property
          -
          -
          -
          +
          -
          +
          -
          \ No newline at end of file +
          +
          \ No newline at end of file diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/partials/canvas/registration.jsp b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/partials/canvas/registration.jsp index 4a965e41f4..56b32369b2 100644 --- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/partials/canvas/registration.jsp +++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/partials/canvas/registration.jsp @@ -20,7 +20,7 @@

          You are not authorized to access this data flow

          -
          + Request Access