Merge branch 'NIFI-280-take2-tworow' of https://github.com/danbress/incubator-nifi into NIFI-280-P19 This closes #19

This commit is contained in:
joewitt 2015-03-17 13:58:00 -04:00
commit cadfc679d8
111 changed files with 3121 additions and 7044 deletions

View File

@ -0,0 +1,49 @@
<?xml version="1.0"?>
<project
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">
<!-- 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. -->
<modelVersion>4.0.0</modelVersion>
<parent>
<groupId>org.apache.nifi</groupId>
<artifactId>nifi-framework</artifactId>
<version>0.0.2-incubating-SNAPSHOT</version>
</parent>
<artifactId>nifi-documentation</artifactId>
<dependencies>
<dependency>
<groupId>org.apache.nifi</groupId>
<artifactId>nifi-nar-utils</artifactId>
</dependency>
<dependency>
<groupId>org.apache.nifi</groupId>
<artifactId>nifi-api</artifactId>
</dependency>
<dependency>
<groupId>org.apache.nifi</groupId>
<artifactId>nifi-properties</artifactId>
</dependency>
<dependency>
<groupId>commons-io</groupId>
<artifactId>commons-io</artifactId>
</dependency>
<dependency>
<groupId>org.apache.commons</groupId>
<artifactId>commons-lang3</artifactId>
</dependency>
<dependency>
<groupId>org.apache.nifi</groupId>
<artifactId>nifi-processor-utils</artifactId>
<scope>test</scope>
</dependency>
</dependencies>
</project>

View File

@ -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;
}

View File

@ -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<Class> 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<? extends ConfigurableComponent> 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<? extends ConfigurableComponent> 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<? extends ConfigurableComponent> 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<? extends ConfigurableComponent> componentClass) {
if (Processor.class.isAssignableFrom(componentClass)) {
return new ProcessorInitializer();
} else if (ControllerService.class.isAssignableFrom(componentClass)) {
return new ControllerServiceInitializer();
} else if (ReportingTask.class.isAssignableFrom(componentClass)) {
return new ReportingTaskingInitializer();
}
return null;
}
/**
* 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;
}
}

View File

@ -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;
}

View File

@ -0,0 +1,417 @@
/*
* 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.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.controller.ControllerService;
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";
@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("<!DOCTYPE html>");
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));
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("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");
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 {
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<PropertyDescriptor> 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 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.");
}
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();
} 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();
}
/**
* 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();
}
xmlStreamWriter.writeEndElement();
} else if (property.getControllerServiceDefinition() != null) {
Class<? extends ControllerService> 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: &lt;p&gt;text&lt;/p&gt;
*
* @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: &lt;p&gt;text&lt;/p&gt;
*
* @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();
}
}

View File

@ -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.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;
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();
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.");
}
}
}

View File

@ -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());
}
}

View File

@ -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());
}
}

View File

@ -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());
}
}

View File

@ -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();
}
}

View File

@ -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<String> getControllerServiceIdentifiers(Class<? extends ControllerService> serviceType)
throws IllegalArgumentException {
return Collections.emptySet();
}
}

View File

@ -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();
}
}

View File

@ -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;
}
}

View File

@ -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<PropertyDescriptor> properties;
static {
List<PropertyDescriptor> props = new ArrayList<>();
props.add(KEYSTORE);
props.add(KEYSTORE_PASSWORD);
props.add(KEYSTORE_TYPE);
properties = Collections.unmodifiableList(props);
}
@Override
protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
return properties;
}
}

View File

@ -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.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.AllowableValue;
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(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)
.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 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")
.description("Failing files").build();
private List<PropertyDescriptor> properties;
private Set<Relationship> relationships;
@Override
protected void init(ProcessorInitializationContext context) {
final List<PropertyDescriptor> properties = new ArrayList<>();
properties.add(DIRECTORY);
properties.add(RECURSE);
properties.add(POLLING_INTERVAL);
properties.add(OPTIONAL_PROPERTY);
properties.add(TYPE_PROPERTY);
properties.add(SERVICE_PROPERTY);
this.properties = Collections.unmodifiableList(properties);
final Set<Relationship> relationships = new HashSet<>();
relationships.add(REL_SUCCESS);
relationships.add(REL_FAILURE);
this.relationships = Collections.unmodifiableSet(relationships);
}
@Override
protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
return properties;
}
@Override
public Set<Relationship> 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();
}
}

View File

@ -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<PropertyDescriptor> getSupportedPropertyDescriptors() {
final List<PropertyDescriptor> descriptors = new ArrayList<>();
descriptors.add(SHOW_DELTAS);
return descriptors;
}
@Override
public void onTrigger(ReportingContext context) {
// TODO Auto-generated method stub
}
}

View File

@ -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
}
}

View File

@ -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();
}

View File

@ -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.documentation.html;
import java.io.ByteArrayOutputStream;
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;
import static org.apache.nifi.documentation.html.XmlValidator.assertContains;
public class HtmlDocumentationWriterTest {
@Test
public void testDocumentControllerService() throws InitializationException, IOException {
ControllerService controllerService = new FullyDocumentedControllerService();
controllerService.initialize(new MockControllerServiceInitializationContext());
DocumentationWriter writer = new HtmlDocumentationWriter();
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
public void testDocumentReportingTask() throws InitializationException, IOException {
ReportingTask reportingTask = new FullyDocumentedReportingTask();
reportingTask.initialize(new MockReportingInitializationContext());
DocumentationWriter writer = new HtmlDocumentationWriter();
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");
}
}

View File

@ -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.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.example.NakedProcessor;
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());
XmlValidator.assertXmlValid(results);
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());
assertContains(results, "Controller Service: ");
assertContains(results, "SampleService");
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...");
}
@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.");
}
}

View File

@ -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));
}
}

View File

@ -31,6 +31,11 @@
<groupId>org.apache.nifi</groupId> <groupId>org.apache.nifi</groupId>
<artifactId>nifi-properties</artifactId> <artifactId>nifi-properties</artifactId>
<scope>compile</scope> <scope>compile</scope>
</dependency>
<dependency>
<groupId>org.apache.nifi</groupId>
<artifactId>nifi-documentation</artifactId>
<scope>compile</scope>
</dependency> </dependency>
<dependency> <dependency>
<groupId>org.slf4j</groupId> <groupId>org.slf4j</groupId>

View File

@ -31,6 +31,7 @@ import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicLong;
import org.apache.nifi.documentation.DocGenerator;
import org.apache.nifi.nar.ExtensionManager; import org.apache.nifi.nar.ExtensionManager;
import org.apache.nifi.nar.ExtensionMapping; import org.apache.nifi.nar.ExtensionMapping;
import org.apache.nifi.nar.NarClassLoaders; import org.apache.nifi.nar.NarClassLoaders;
@ -118,6 +119,8 @@ public class NiFi {
ExtensionManager.discoverExtensions(); ExtensionManager.discoverExtensions();
ExtensionManager.logClassLoaderMapping(); ExtensionManager.logClassLoaderMapping();
DocGenerator.generate(properties);
// load the server from the framework classloader // load the server from the framework classloader
Thread.currentThread().setContextClassLoader(frameworkClassLoader); Thread.currentThread().setContextClassLoader(frameworkClassLoader);
Class<?> jettyServer = Class.forName("org.apache.nifi.web.server.JettyServer", true, frameworkClassLoader); Class<?> jettyServer = Class.forName("org.apache.nifi.web.server.JettyServer", true, frameworkClassLoader);

View File

@ -42,30 +42,75 @@ body {
/* tables */ /* tables */
table { table {
background-color: #fefefe; color:#666;
border: 1px solid #ccc;
border-left: 6px solid #ccc;
color: #555;
display: block;
margin-bottom: 12px;
padding: 5px 8px;
}
tr td {
font-size:14px; font-size:14px;
vertical-align:top; background:#eaebec;
text-align:left; border:#ccc 1px solid;
padding: 4px; -webkit-border-radius:3px;
border-width: 0; border-radius:3px;
width: 100%;
} }
tr th { table th {
font-size: 16px; padding:11px 15px 12px 15px;
vertical-align:top; border-top:1px solid #fafafa;
border-bottom:1px solid #e0e0e0;
background: #ededed;
}
table th:first-child {
text-align: left; text-align: left;
padding: 4px; padding-left:10px;
border-width: 0; }
white-space: nowrap;
table th:last-child {
text-align: left;
padding-left:10px;
}
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:10px;
}
table td:first-child {
text-align: left;
padding-left:10px;
border-left: 0;
}
table td:last-child {
text-align: left;
padding-left:10px;
border-left: 0;
vertical-align: top;
}
table td {
padding:12px;
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 */ /* links */
@ -99,10 +144,13 @@ p strong {
} }
/* ul li */ /* ul li */
td ul {
margin: 0px 0px 0px 0px;
padding-left: 20px;
}
ul li { ul li {
font-family: 'Noto Serif', 'DejaVu Serif', serif; text-align: left;
font-size: 16px; display: list-item;
} }
ul li strong { ul li strong {

View File

@ -39,6 +39,7 @@
<module>nifi-administration</module> <module>nifi-administration</module>
<module>nifi-web</module> <module>nifi-web</module>
<module>nifi-resources</module> <module>nifi-resources</module>
<module>nifi-documentation</module>
</modules> </modules>
<dependencies> <dependencies>
<dependency> <dependency>

View File

@ -108,6 +108,11 @@
<artifactId>nifi-web-security</artifactId> <artifactId>nifi-web-security</artifactId>
<version>0.1.0-incubating-SNAPSHOT</version> <version>0.1.0-incubating-SNAPSHOT</version>
</dependency> </dependency>
<dependency>
<groupId>org.apache.nifi</groupId>
<artifactId>nifi-documentation</artifactId>
<version>0.0.2-incubating-SNAPSHOT</version>
</dependency>
</dependencies> </dependencies>
</dependencyManagement> </dependencyManagement>
</project> </project>

View File

@ -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 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. issues if there are too many concurrent tasks and the flow file sizes are large.
<h2>Properties:</h2>
<p>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.</p>
<ul>
<li>Hadoop Configuration Resources
<ul>
<li>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.</li>
<li>Default value: none</li>
</ul>
</li>
<li>compression type
<ul>
<li>Type of compression to use when creating Sequence File.</li>
<li>Default value: none</li>
</ul>
</li>
</ul>
<h2>Relationships:</h2>
<ul>
<li>success
<ul>
<li>Generated Sequence Files are sent to this relationship.</li>
</ul>
</li>
<li>fail
<ul>
<li>Incoming files that failed to generate a Sequence File are sent to this relationship.</li>
</ul>
</li>
</ul>
<p> <p>
<strong>See Also:</strong> <strong>See Also:</strong>
</p> </p>

View File

@ -0,0 +1,57 @@
<!DOCTYPE html>
<html lang="en">
<!--
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.
-->
<head>
<meta charset="utf-8" />
<title>GetHDFS</title>
<link rel="stylesheet" href="../../css/component-usage.css" type="text/css" />
</head>
<body>
<p>
<strong>Modifies Attributes:</strong>
</p>
<table>
<thead>
<tr>
<th>Attribute Name</th>
<th>Description</th>
</tr>
</thead>
<tbody>
<tr>
<td>filename</td>
<td>The name of the file that was read from HDFS.</td>
</tr>
<tr>
<td>path</td>
<td>The path is set to the relative path of the file's directory on HDFS. For example, if the <em>Directory</em>
property is set to <code>/tmp</code>, then files picked up from /tmp will have the path attribute set to
&quot;<code>./</code>&quot;. If the <em>Recurse Subdirectories</em> property is set to <code>true</code>
and a file is picked up from /tmp/abc/1/2/3, then the path attribute will be set to &quot;<code>abc/1/2/3</code>&quot;.
</td>
</tr>
</tbody>
</table>
<p>
<strong>See Also:</strong>
</p>
<ul>
<li><a href="../org.apache.nifi.processors.hadoop.PutHDFS/index.html">PutHDFS</a></li>
</ul>
</body>
</html>

View File

@ -1,162 +0,0 @@
<!DOCTYPE html>
<html lang="en">
<!--
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.
-->
<head>
<meta charset="utf-8" />
<title>GetHDFS</title>
<link rel="stylesheet" href="../../css/component-usage.css" type="text/css" />
</head>
<body>
<!-- Processor Documentation ================================================== -->
<h2>Description:</h2>
<p>This processor reads files from an HDFS cluster into NiFi FlowFiles.</p>
<p>
<strong>Modifies Attributes:</strong>
</p>
<table border="1">
<thead>
<tr>
<th>Attribute Name</th>
<th>Description</th>
</tr>
</thead>
<tbody>
<tr>
<td>filename</td>
<td>The name of the file that was read from HDFS.</td>
</tr>
<tr>
<td>path</td>
<td>The path is set to the relative path of the file's directory on HDFS. For example, if the <em>Directory</em>
property is set to <code>/tmp</code>, then files picked up from /tmp will have the path attribute set to
&quot;<code>./</code>&quot;. If the <em>Recurse Subdirectories</em> property is set to <code>true</code>
and a file is picked up from /tmp/abc/1/2/3, then the path attribute will be set to &quot;<code>abc/1/2/3</code>&quot;.
</td>
</tr>
</tbody>
</table>
<h2>Properties:</h2>
<p>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.</p>
<ul>
<li>Hadoop Configuration Resources
<ul>
<li>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.</li>
<li>Default value: none</li>
</ul>
</li>
<li><strong>Directory</strong>
<ul>
<li>The HDFS directory from which FlowFile content should be read.</li>
<li>Default value: none</li>
</ul></li>
<li><strong>Recurse Subdirectories</strong>
<ul>
<li>A Boolean value (true/false), when true will pull files from subdirectories of the HDFS <em>Directory</em>.
</li>
<li>Default value: true</li>
</ul></li>
<li><strong>Keep Source File</strong>
<ul>
<li>A Boolean value (true/false), indicates whether to keep (true) or delete (false) the file from HDFS
after it has been successfully transferred.</li>
<li>Default value: false</li>
</ul></li>
<li>File Filter Regex
<ul>
<li>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.</li>
<li>Default value: none</li>
</ul>
</li>
<li><strong>Filter Match Name Only</strong>
<ul>
<li>A Boolean value (true/false), when true <em>File Filter Regex</em> will match on just the filename,
otherwise subdirectory names will be included with filename in the regex comparison.
</li>
<li>Default value: true</li>
</ul></li>
<li><strong>Ignore Dotted Files</strong>
<ul>
<li>A Boolean value (true/false), when true files whose names begin with a dot (&quot;.&quot;) will not
be fetched.</li>
<li>Default value: true</li>
</ul></li>
<li><strong>Minimum File Age</strong>
<ul>
<li>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.</li>
<li>Default value: 0 sec</li>
</ul></li>
<li>Maximum File Age
<ul>
<li>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.</li>
<li>Default value: none</li>
</ul>
</li>
<li><strong>Polling Interval</strong>
<ul>
<li>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.</li>
<li>Default value: 0 sec</li>
</ul></li>
<li><strong>Batch Size</strong>
<ul>
<li>The maximum number of files to pull in each iteration, based on configured run schedule.</li>
<li>Default value: 100</li>
</ul></li>
<li>IO Buffer Size
<ul>
<li>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.</li>
<li>Default value: none</li>
</ul>
</li>
</ul>
<h2>Relationships:</h2>
<ul>
<li>success
<ul>
<li>All files retrieved from HDFS are transferred to this relationship.</li>
</ul>
</li>
<li>passthrough
<ul>
<li>If this processor has an input queue for some reason, then FlowFiles arriving on that input are
transferred to this relationship.</li>
</ul>
</li>
</ul>
<p>
<strong>See Also:</strong>
</p>
<ul>
<li><a href="../org.apache.nifi.processors.hadoop.PutHDFS/index.html">PutHDFS</a></li>
</ul>
</body>
</html>

View File

@ -0,0 +1,32 @@
<!DOCTYPE html>
<html lang="en">
<!--
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.
-->
<head>
<meta charset="utf-8" />
<title>GetHDFSSequenceFile</title>
<link rel="stylesheet" href="../../css/component-usage.css" type="text/css" />
</head>
<body>
<p>
<strong>See Also:</strong>
</p>
<ul>
<li><a href="../org.apache.nifi.processors.hadoop.PutHDFS/index.html">PutHDFS</a></li>
</ul>
</body>
</html>

View File

@ -1,150 +0,0 @@
<!DOCTYPE html>
<html lang="en">
<!--
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.
-->
<head>
<meta charset="utf-8" />
<title>GetHDFSSequenceFile</title>
<link rel="stylesheet" href="../../css/component-usage.css" type="text/css" />
</head>
<body>
<!-- Processor Documentation ================================================== -->
<h2>Description:</h2>
<p>This processor is used to pull files from HDFS. The files being pulled in MUST be <code>SequenceFile</code>
formatted files. The processor creates a flow file for each key/value entry in the ingested <code>SequenceFile</code>.
The created flow file's content depends on the value of the optional configuration property <b>FlowFile Content</b>. Currently,
there are two choices: VALUE ONLY and KEY VALUE PAIR. With the prior, only the <code>SequenceFile</code> value element is
written to the flow file contents. With the latter, the <code>SequenceFile</code> 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.</p>
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.
<h2>Properties:</h2>
<p>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.</p>
<ul>
<li>Hadoop Configuration Resources
<ul>
<li>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.</li>
<li>Default value: none</li>
</ul>
</li>
<li><strong>FlowFile Content</strong>
<ul>
<li>Indicate if the content is to be both the key and value of the Sequence File, or just the value.</li>
<li>Default value: VALUE ONLY</li>
</ul>
</li>
<li><strong>Directory</strong>
<ul>
<li>The HDFS directory from which FlowFile content should be read.</li>
<li>Default value: none</li>
</ul></li>
<li><strong>Recurse Subdirectories</strong>
<ul>
<li>A Boolean value (true/false), when true will pull files from subdirectories of the HDFS <em>Directory</em>.
</li>
<li>Default value: true</li>
</ul></li>
<li><strong>Keep Source File</strong>
<ul>
<li>A Boolean value (true/false), indicates whether to keep (true) or delete (false) the file from HDFS
after it has been successfully transferred.</li>
<li>Default value: false</li>
</ul></li>
<li>File Filter Regex
<ul>
<li>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.</li>
<li>Default value: none</li>
</ul>
</li>
<li><strong>Filter Match Name Only</strong>
<ul>
<li>A Boolean value (true/false), when true <em>File Filter Regex</em> will match on just the filename,
otherwise subdirectory names will be included with filename in the regex comparison.
</li>
<li>Default value: true</li>
</ul></li>
<li><strong>Ignore Dotted Files</strong>
<ul>
<li>A Boolean value (true/false), when true files whose names begin with a dot (&quot;.&quot;) will not
be fetched.</li>
<li>Default value: true</li>
</ul></li>
<li><strong>Minimum File Age</strong>
<ul>
<li>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.</li>
<li>Default value: 0 sec</li>
</ul></li>
<li>Maximum File Age
<ul>
<li>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.</li>
<li>Default value: none</li>
</ul>
</li>
<li><strong>Polling Interval</strong>
<ul>
<li>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.</li>
<li>Default value: 0 sec</li>
</ul></li>
<li><strong>Batch Size</strong>
<ul>
<li>The maximum number of files to pull in each iteration, based on configured run schedule.</li>
<li>Default value: 100</li>
</ul></li>
<li>IO Buffer Size
<ul>
<li>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.</li>
<li>Default value: none</li>
</ul>
</li>
</ul>
<h2>Relationships:</h2>
<ul>
<li>success
<ul>
<li>All files retrieved from HDFS are transferred to this relationship.</li>
</ul>
</li>
<li>passthrough
<ul>
<li>If this processor has an input queue for some reason, then FlowFiles arriving on that input are
transferred to this relationship.</li>
</ul>
</li>
</ul>
<p>
<strong>See Also:</strong>
</p>
<ul>
<li><a href="../org.apache.nifi.processors.hadoop.PutHDFS/index.html">PutHDFS</a></li>
</ul>
</body>
</html>

View File

@ -0,0 +1,51 @@
<!DOCTYPE html>
<html lang="en">
<!--
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.
-->
<head>
<meta charset="utf-8" />
<title>PutHDFS</title>
<link rel="stylesheet" href="../../css/component-usage.css" type="text/css" />
</head>
<body>
<p>
<strong>Uses Attributes:</strong>
</p>
<table>
<thead>
<tr>
<th>Attribute Name</th>
<th>Description</th>
</tr>
</thead>
<tbody>
<tr>
<td>filename</td>
<td>The name of the file written to HDFS comes from the value of this attribute.</td>
</tr>
</tbody>
</table>
<p>
<strong>See Also:</strong>
</p>
<ul>
<li><a href="../org.apache.nifi.processors.hadoop.GetHDFS/index.html">GetHDFS</a></li>
</ul>
</body>
</html>

View File

@ -1,159 +0,0 @@
<!DOCTYPE html>
<html lang="en">
<!--
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.
-->
<head>
<meta charset="utf-8" />
<title>PutHDFS</title>
<link rel="stylesheet" href="../../css/component-usage.css" type="text/css" />
</head>
<body>
<!-- Processor Documentation ================================================== -->
<h2>Description:</h2>
<p>
This processor writes FlowFiles to an HDFS cluster. It will create directories in which to store files as
needed based on the <em>Directory</em> property.
</p>
<p>
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 <em>Remote Owner</em> property, the group can be overridden using the <em>Remote Group</em>
property, and the read/write/execute permissions can be overridden using the <em>Permissions umask</em> property.
</p>
<p>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.</p>
<p>
NOTE: The <em>Permissions umask</em> property cannot add execute permissions to regular files.
</p>
<p>
<strong>Uses Attributes:</strong>
</p>
<table border="1">
<thead>
<tr>
<th>Attribute Name</th>
<th>Description</th>
</tr>
</thead>
<tbody>
<tr>
<td>filename</td>
<td>The name of the file written to HDFS comes from the value of this attribute.</td>
</tr>
</tbody>
</table>
<h2>Properties:</h2>
<p>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.</p>
<ul>
<li>Hadoop Configuration Resources
<ul>
<li>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.</li>
<li>Default value: none</li>
</ul>
</li>
<li><strong>Directory</strong>
<ul>
<li>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}.</li>
<li>Default value: none</li>
<li>Supports expression language: true</li>
</ul></li>
<li><strong>Conflict Resolution Strategy</strong>
<ul>
<li>Indicates what should happen when a file with the same name already exists in the output directory.
Valid options are:</li>
<ul>
<li>replace - existing file is overwritten by new file</li>
<li>ignore - existing file is untouched, FlowFile routed to success</li>
<li>fail - existing file is untouched, FlowFile routed to failure</li>
</ul>
<li>Default value: fail</li>
</ul></li>
<li>Block Size
<ul>
<li>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.</li>
<li>Default value: none</li>
</ul>
</li>
<li>IO Buffer Size
<ul>
<li>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.</li>
<li>Default value: none</li>
</ul>
</li>
<li>Replication
<ul>
<li>Number of times that HDFS will replicate each file. This must be an integer greater than 0. This
overrides the Hadoop Configuration.</li>
<li>Default value: none</li>
</ul>
</li>
<li>Permissions umask
<ul>
<li>A umask represented as an octal number which determines the permissions of files written to HDFS.
This overrides the Hadoop Configuration dfs.umaskmode.</li>
<li>Default value: none</li>
</ul>
</li>
<li>Remote Owner
<ul>
<li>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.</li>
<li>Default value: none</li>
</ul>
</li>
<li>Remote Group
<ul>
<li>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.</li>
<li>Default value: none</li>
</ul>
</li>
</ul>
<h2>Relationships:</h2>
<ul>
<li>success
<ul>
<li>Files that have been successfully written to HDFS are transferred to this relationship.</li>
</ul>
</li>
<li>failure
<ul>
<li>Files that could not be written to HDFS for some reason are transferred to this relationship.</li>
</ul>
</li>
</ul>
<p>
<strong>See Also:</strong>
</p>
<ul>
<li><a href="../org.apache.nifi.processors.hadoop.GetHDFS/index.html">GetHDFS</a></li>
</ul>
</body>
</html>

View File

@ -0,0 +1,78 @@
<!DOCTYPE html>
<html lang="en">
<!--
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.
-->
<head>
<meta charset="utf-8" />
<title>GetKafka</title>
<link rel="stylesheet" href="../../css/component-usage.css" type="text/css" />
</head>
<body>
<!-- Processor Documentation ================================================== -->
<h2>Description:</h2>
<p>
This Processors polls <a href="http://kafka.apache.org/">Apache Kafka</a>
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 <code>kafka.key</code>
will be added to the FlowFile, with the value being the UTF-8 Encoded value
of the Message's Key.
</p>
<p>
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.
</p>
<p>
<strong>Modifies Attributes:</strong>
</p>
<table>
<thead>
<tr>
<th>Attribute Name</th>
<th>Description</th>
</tr>
</thead>
<tbody>
<tr>
<td>kafka.topic</td>
<td>The name of the Kafka Topic from which the message was received</td>
</tr>
<tr>
<td>kafka.key</td>
<td>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.</td>
</tr>
<tr>
<td>kafka.partition</td>
<td>The partition of the Kafka Topic from which the message was received. This attribute is added only
if the batch size is 1.</td>
</tr>
<tr>
<td>kafka.offset</td>
<td>The offset of the message within the Kafka partition. This attribute is added only
if the batch size is 1.</td>
</tr>
</tbody>
</table>
</body>
</html>

View File

@ -1,173 +0,0 @@
<!DOCTYPE html>
<html lang="en">
<!--
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.
-->
<head>
<meta charset="utf-8" />
<title>GetKafka</title>
<link rel="stylesheet" href="../../css/component-usage.css" type="text/css" />
</head>
<body>
<!-- Processor Documentation ================================================== -->
<h2>Description:</h2>
<p>
This Processors polls <a href="http://kafka.apache.org/">Apache Kafka</a>
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 <code>kafka.key</code>
will be added to the FlowFile, with the value being the UTF-8 Encoded value
of the Message's Key.
</p>
<p>
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.
</p>
<p>
<strong>Modifies Attributes:</strong>
</p>
<table border="1">
<thead>
<tr>
<th>Attribute Name</th>
<th>Description</th>
</tr>
</thead>
<tbody>
<tr>
<td>kafka.topic</td>
<td>The name of the Kafka Topic from which the message was received</td>
</tr>
<tr>
<td>kafka.key</td>
<td>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.</td>
</tr>
<tr>
<td>kafka.partition</td>
<td>The partition of the Kafka Topic from which the message was received. This attribute is added only
if the batch size is 1.</td>
</tr>
<tr>
<td>kafka.offset</td>
<td>The offset of the message within the Kafka partition. This attribute is added only
if the batch size is 1.</td>
</tr>
</tbody>
</table>
<p>
<strong>Properties:</strong>
</p>
<p>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.</p>
<ul>
<li><strong>ZooKeeper Connection String</strong>
<ul>
<li>The Connection String to use in order to connect to ZooKeeper. This is often a
comma-separated list of &lt;host&gt;:&lt;port&gt; combinations. For example,
host1:2181,host2:2181,host3:2188</li>
<li>Default value: no default</li>
<li>Supports expression language: false</li>
</ul>
</li>
<li><strong>Topic Name</strong>
<ul>
<li>The Kafka Topic to pull messages from</li>
<li>Default value: no default</li>
<li>Supports expression language: false</li>
</ul>
</li>
<li><strong>Zookeeper Commit Frequency</strong>
<ul>
<li>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
</li>
<li>Default value: 60 secs</li>
<li>Supports expression language: false</li>
</ul>
</li>
<li><strong>ZooKeeper Communications Timeout</strong>
<ul>
<li>The amount of time to wait for a response from ZooKeeper before determining that there is a communications error</li>
<li>Default value: 30 secs</li>
<li>Supports expression language: false</li>
</ul>
</li>
<li><strong>Kafka Communications Timeout</strong>
<ul>
<li>The amount of time to wait for a response from Kafka before determining that there is a communications error</li>
<li>Default value: 30 secs</li>
<li>Supports expression language: false</li>
</ul>
</li>
<li><strong>Batch Size</strong>
<ul>
<li>Specifies the maximum number of messages to combine into a single FlowFile.
These messages will be concatenated together with the &lt;Message Demarcator&gt;
string placed between the content of each message. If the messages from Kafka
should not be concatenated together, leave this value at 1.</li>
<li>Default value: 1</li>
<li>Supports expression language: false</li>
</ul>
</li>
<li><strong>Message Demarcator</strong>
<ul>
<li>Specifies the characters to use in order to demarcate multiple messages from Kafka.
If the &lt;Batch Size&gt; 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.
</li>
<li>Default value: \n</li>
<li>Supports expression language: false</li>
</ul>
</li>
<li><strong>Client Name</strong>
<ul>
<li>Client Name to use when communicating with Kafka</li>
<li>Default value: "NiFi-" followed by the UUID of the Processor</li>
<li>Supports expression language: false</li>
</ul>
</li>
</ul>
<p>
<strong>Relationships:</strong>
</p>
<ul>
<li>success
<ul>
<li>All messages that are received from Kafka are routed to the 'success' relationship</li>
</ul>
</li>
</ul>
</body>
</html>

View File

@ -16,8 +16,7 @@
--> -->
<head> <head>
<meta charset="utf-8" /> <meta charset="utf-8" />
<title>ValidateXml</title> <title>PutKafka</title>
<link rel="stylesheet" href="../../css/component-usage.css" type="text/css" /> <link rel="stylesheet" href="../../css/component-usage.css" type="text/css" />
</head> </head>
@ -25,32 +24,22 @@
<!-- Processor Documentation ================================================== --> <!-- Processor Documentation ================================================== -->
<h2>Description:</h2> <h2>Description:</h2>
<p> <p>
This processor validates the contents of FlowFiles against a user-specified XML schema file. This Processors puts the contents of a FlowFile to a Topic in
<p> <a href="http://kafka.apache.org/">Apache Kafka</a>. The full contents of
a FlowFile becomes the contents of a single message in Kafka.
<strong>Properties:</strong> This message is optionally assigned a key by using the
&lt;Kafka Key&gt; Property.
</p> </p>
<p>
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.
</p>
<ul>
<li><strong>Schema File</strong>
<ul>
<li>The local file path to the schema file to be used for validation.</li>
<li>Default value: no default</li>
<li>Supports expression language: false</li>
</ul></li>
</ul>
<p> <p>
<strong>Relationships:</strong> 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'.
</p> </p>
<ul>
<li>valid
<ul>
<li>If FlowFiles are successfully validated against the schema, then they follow this relationship.</li>
</ul></li>
</ul>
</body> </body>
</html> </html>

View File

@ -1,189 +0,0 @@
<!DOCTYPE html>
<html lang="en">
<!--
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.
-->
<head>
<meta charset="utf-8" />
<title>PutKafka</title>
<link rel="stylesheet" href="../../css/component-usage.css" type="text/css" />
</head>
<body>
<!-- Processor Documentation ================================================== -->
<h2>Description:</h2>
<p>
This Processors puts the contents of a FlowFile to a Topic in
<a href="http://kafka.apache.org/">Apache Kafka</a>. 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
&lt;Kafka Key&gt; Property.
</p>
<p>
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'.
</p>
<p>
<strong>Properties:</strong>
</p>
<p>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.</p>
<ul>
<li><strong>Known Brokers</strong>
<ul>
<li>
A comma-separated list of known Kafka Brokers in the format
&lt;host&gt;:&lt;port&gt;. This list does not need to be
exhaustive but provides a mechanism for determining which
other nodes belong to the Kafka cluster.
</li>
<li>Default value: no default</li>
<li>Supports expression language: false</li>
</ul>
</li>
<li><strong>Topic Name</strong>
<ul>
<li>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.
</li>
<li>Default value: no default</li>
<li>Supports expression language: true</li>
</ul>
</li>
<li>Kafka Key
<ul>
<li>
The Key to use for the Message. If no value is given, the message
will not be given a Key.
</li>
<li>Default value: no default</li>
<li>Supports expression language: true</li>
</ul>
</li>
<li><strong>Delivery Guarantee</strong>
<ul>
<li>
Specifies the requirement for guaranteeing that a message is sent to Kafka.
This Property can have one of three different values:
<ul>
<li>
<b>Guarantee Replicated Delivery</b> - FlowFile will be routed to
failure unless the message is replicated to the appropriate number
of Kafka Nodes according to the Topic configuration
</li>
<li>
<b>Guarantee Single Node Delivery</b> - 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
&lt;Guarantee Replicated Delivery&gt; but can result in data loss
if a Kafka node crashes
</li>
<li>
<b>Best Effort</b> - 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.
</li>
</ul>
</li>
<li>Default value: Best Effort</li>
<li>Supports expression language: false</li>
</ul>
</li>
<li>Message Delimiter
<ul>
<li>
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.
</li>
<li>Default value: no default</li>
<li>Supports expression language: true</li>
</ul>
</li>
<li><strong>Communications Timeout</strong>
<ul>
<li>
The amount of time to wait for a response from Kafka before determining
that there is a communications error
</li>
<li>Default value: 30 secs</li>
<li>Supports expression language: false</li>
</ul>
</li>
<li><strong>Max Buffer Size</strong>
<ul>
<li>
The maximum amount of data to buffer in memory before sending to Kafka
</li>
<li>Default value: 1 MB</li>
<li>Supports expression language: false</li>
</ul>
</li>
<li><strong>Client Name</strong>
<ul>
<li>Client Name to use when communicating with Kafka</li>
<li>Default value: "NiFi-" followed by the UUID of the Processor</li>
<li>Supports expression language: false</li>
</ul>
</li>
</ul>
<p>
<strong>Relationships:</strong>
</p>
<ul>
<li>success
<ul>
<li>All FlowFiles that are successfully sent to Kafka are routed
to this relationship. If using the &lt;Message Delimiter&gt; 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.
</li>
</ul>
</li>
<li>failure
<ul>
<li>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.
</li>
</ul>
</li>
</ul>
</body>
</html>

View File

@ -1,63 +0,0 @@
<!DOCTYPE html>
<html lang="en">
<!--
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.
-->
<head>
<meta charset="utf-8" />
<title>EncodeContent</title>
<link rel="stylesheet" href="../../css/component-usage.css" type="text/css" />
</head>
<body>
<!-- Processor Documentation ================================================== -->
<h2>Description:</h2>
<p>
This processor base64 encodes FlowFile content, or decodes FlowFile content from base64.
<p>
<strong>Properties:</strong>
</p>
<p>
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.
</p>
<ul>
<li><strong>Mode</strong>
<ul>
<li>This property specifies whether the content should be base64 encoded, or decoded from base64. Valid values are:
<ul>
<li>Encode</li>
<li>Decode</li>
</ul></li>
<li>Default value: Encode</li>
<li>Supports expression language: false</li>
</ul></li>
</ul>
<p>
<strong>Relationships:</strong>
</p>
<ul>
<li>failure
<ul>
<li>If a FlowFile cannot be encoded or decoded, then it follows this relationship</li>
</ul></li>
<li>success
<ul>
<li>If a FlowFile is successfully encoded or decoded, then it follows this relationship</li>
</ul></li>
</ul>
</body>
</html>

View File

@ -0,0 +1,68 @@
<!DOCTYPE html>
<html lang="en">
<!--
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.
-->
<head>
<meta charset="utf-8" />
<title>CompressContent</title>
<link rel="stylesheet" href="../../css/component-usage.css" type="text/css" />
</head>
<body>
<!-- Processor Documentation ================================================== -->
<p>
<strong>Uses Attributes:</strong>
</p>
<table>
<thead>
<tr>
<th>Attribute Name</th>
<th>Description</th>
</tr>
</thead>
<tbody>
<tr>
<td>mime.type</td>
<td>If the <code>Compression Format</code> is set to <code>use mime.type attribute</code>,
this attribute is used to determine the compression type. Otherwise, this attribute is ignored.</td>
</tr>
</tbody>
</table>
<p>
<strong>Modifies Attributes:</strong>
</p>
<table>
<thead>
<tr>
<th>Attribute Name</th>
<th>Description</th>
</tr>
</thead>
<tbody>
<tr>
<td>mime.type</td>
<td>If the <code>Mode</code> property is set to <code>compress</code>, the appropriate MIME Type is set. If the
<code>Mode</code> property is set to <code>decompress</code> and the file is successfully decompressed,
this attribute is removed, as the MIME Type is no longer known.
</td>
</tr>
</tbody>
</table>
</body>
</html>

View File

@ -1,166 +0,0 @@
<!DOCTYPE html>
<html lang="en">
<!--
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.
-->
<head>
<meta charset="utf-8" />
<title>CompressContent</title>
<link rel="stylesheet" href="../../css/component-usage.css" type="text/css" />
</head>
<body>
<!-- Processor Documentation ================================================== -->
<h2>Description:</h2>
<p>
<p>
This processor compresses and decompresses the contents of
FlowFiles using a user-specified compression algorithm.
</p>
<p>
<strong>Uses Attributes:</strong>
</p>
<table border="1">
<thead>
<tr>
<th>Attribute Name</th>
<th>Description</th>
</tr>
</thead>
<tbody>
<tr>
<td>mime.type</td>
<td>If the <code>Compression Format</code> is set to <code>use mime.type attribute</code>,
this attribute is used to determine the compression type. Otherwise, this attribute is ignored.</td>
</tr>
</tbody>
</table>
<p>
<strong>Modifies Attributes:</strong>
</p>
<table border="1">
<thead>
<tr>
<th>Attribute Name</th>
<th>Description</th>
</tr>
</thead>
<tbody>
<tr>
<td>mime.type</td>
<td>If the <code>Mode</code> property is set to <code>compress</code>, the appropriate MIME Type is set. If the
<code>Mode</code> property is set to <code>decompress</code> and the file is successfully decompressed,
this attribute is removed, as the MIME Type is no longer known.
</td>
</tr>
</tbody>
</table>
<p>
<strong>Properties:</strong>
</p>
<p>
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.
</p>
<ul>
<li>
<strong>Mode</strong>
<ul>
<li>
Indicates whether the processor should compress or decompress
content. Valid values are:
<ul>
<li>compress</li>
<li>decompress</li>
</ul>
</li>
<li>Default value: compress</li>
<li>Supports expression language: false</li>
</ul>
</li>
<li>
<strong>Compression Format</strong>
<ul>
<li>
The compression format to use. Valid values are:
<ul>
<li>gzip : GZIP compression format</li>
<li>bzip2 : BZIP2 compression format</li>
<li>xz-lzma2 : XZ-LZMA2 compression format</li>
<li>lzma : LZMA compression format</li>
<li>
use mime.type attribute : The compression format should be
determined by examining the
<code>mime.type</code>
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
<a href="../org.apache.nifi.processors.standard.IdentifyMimeType/index.html">IdentifiyMimeType Processor</a>
so that files that are
compressed can automatically be decompressed and files that are not
compressed will continue on.
</li>
</ul>
</li>
<li>Default value: use mime.type attribute</li>
<li>Supports expression language: false</li>
</ul>
</li>
<li>
<strong>Compression Level</strong>
<ul>
<li>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.</li>
<li>Default value: 1</li>
<li>Supports expression language: false.</li>
</ul>
</li>
</ul>
<p>
<strong>Relationships:</strong>
</p>
<ul>
<li>
failure
<ul>
<li>If FlowFiles fail to compress or decompress, then they follow
this relationship.</li>
</ul>
</li>
<li>
success
<ul>
<li>If FlowFiles are successfully compressed or decompressed, then
they follow this relationship.</li>
</ul>
</li>
</ul>
</body>
</html>

View File

@ -1,116 +0,0 @@
<!DOCTYPE html>
<html lang="en">
<!--
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.
-->
<head>
<meta charset="utf-8" />
<title>ControlRate</title>
<link rel="stylesheet" href="../../css/component-usage.css" type="text/css" />
</head>
<body>
<!-- Processor Documentation ================================================== -->
<h2>Description:</h2>
<p>This processor controls the rate at which data is
transferred to follow-on processors.
</p>
<p>
<strong>Properties:</strong>
</p>
<p>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.</p>
<ul>
<li><strong>Rate Control Criteria</strong>
<ul>
<li>Indicates the criteria used to control the throughput rate.
Changing this value resets the rate counters. Possible values are:
<ul>
<li>data rate</li>
<li>flowfile count</li>
<li>attribute value</li>
</ul>
</li>
<li>Default value: data rate</li>
<li>Supports expression language: false</li>
</ul></li>
<li><strong>Maximum Rate</strong>
<ul>
<li>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).</li>
<li>Default value: no default</li>
<li>Supports expression language: false</li>
</ul></li>
<li>Rate Controlled Attribute
<ul>
<li>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.</li>
<li>Default value: no default</li>
<li>Supports expression language: true.</li>
</ul>
</li>
<li><strong>Time Duration</strong>
<ul>
<li>The amount of time to which the Maximum Rate property
pertains. Changing this value resets the rate counters.</li>
<li>Default value: 1 min</li>
<li>Supports expression language: false.</li>
</ul></li>
<li>Grouping Attribute
<ul>
<li>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.</li>
<li>Default value: no default</li>
<li>Supports expression language: true.</li>
</ul>
</li>
</ul>
<p>
<strong>Relationships:</strong>
</p>
<ul>
<li>success
<ul>
<li>If FlowFiles are successfully processed, then they follow
this relationship.</li>
</ul>
</li>
</ul>
</body>
</html>

View File

@ -1,65 +0,0 @@
<!DOCTYPE html>
<html lang="en">
<!--
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.
-->
<head>
<meta charset="utf-8" />
<title>ConvertCharacterSet</title>
<link rel="stylesheet" href="../../css/component-usage.css" type="text/css" />
</head>
<body>
<!-- Processor Documentation ================================================== -->
<h2>Description:</h2>
<p>This processor converts a FlowFile's content from one
character set to another.</p>
<p>
<strong>Properties:</strong>
</p>
<p>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.</p>
<ul>
<li><strong>Input Character Set</strong>
<ul>
<li>The name of the character set to expect for the input.</li>
<li>Default value: no default</li>
<li>Supports expression language: false</li>
</ul></li>
<li><strong>Output Character Set</strong>
<ul>
<li>The name of the character set to convert to</li>
<li>Default value: no default</li>
<li>Supports expression language: false</li>
</ul></li>
</ul>
<p>
<strong>Relationships:</strong>
</p>
<ul>
<li>success
<ul>
<li>If a FlowFile's character set is successfully converted,
then it follows this relationship.</li>
</ul>
</li>
</ul>
</body>
</html>

View File

@ -0,0 +1,58 @@
<!DOCTYPE html>
<html lang="en">
<!--
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.
-->
<head>
<meta charset="utf-8" />
<title>DetectDuplicate</title>
<link rel="stylesheet" href="../../css/component-usage.css" type="text/css" />
</head>
<body>
<p>
<strong>Modifies Attributes:</strong>
</p>
<table>
<thead>
<tr>
<th>Attribute Name</th>
<th>Description</th>
</tr>
</thead>
<tbody>
<tr>
<td>original.flowfile.description</td>
<td>All FlowFiles routed to the <code>duplicate</code>
relationship will have an attribute added named <code>original.flowfile.description</code>.
The value of this attribute is determined by the attributes of the
original copy of the data and by the FlowFile Description property
</td>
</tr>
</tbody>
</table>
<p>
<strong>See Also:</strong>
</p>
<ul>
<li><a href="../org.apache.nifi.distributed.cache.client.DistributedMapCacheClientService/index.html">Distributed Map Cache Client Service</a></li>
<li><a href="../org.apache.nifi.distributed.cache.server.map.DistributedMapCacheServer/index.html">Distributed Map Cache Server</a></li>
</ul>
</body>
</html>

View File

@ -1,147 +0,0 @@
<!DOCTYPE html>
<html lang="en">
<!--
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.
-->
<head>
<meta charset="utf-8" />
<title>DetectDuplicate</title>
<link rel="stylesheet" href="../../css/component-usage.css" type="text/css" />
</head>
<body>
<h2>Description:</h2>
<p>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.</p>
<p>
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.
</p>
<p>
<strong>Modifies Attributes:</strong>
</p>
<table border="1">
<thead>
<tr>
<th>Attribute Name</th>
<th>Description</th>
</tr>
</thead>
<tbody>
<tr>
<td>original.flowfile.description</td>
<td>All FlowFiles routed to the <code>duplicate</code>
relationship will have an attribute added named <code>original.flowfile.description</code>.
The value of this attribute is determined by the attributes of the
original copy of the data and by the FlowFile Description property
</td>
</tr>
</tbody>
</table>
<p>
<strong>Properties:</strong>
</p>
<p>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.</p>
<ul>
<li><strong>Distributed Cache Service</strong>
<ul>
<li>The Controller Service that is used to cache unique
identifiers which are used to determine duplicates</li>
<li>Default value: (None)</li>
<li>Supports expression language: false</li>
</ul></li>
<li><strong>Cache Entry Identifier</strong>
<ul>
<li>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</li>
<li>Default value: ${hash.value}</li>
<li>Supports expression language: true</li>
</ul></li>
<li><strong>FlowFile Description</strong>
<ul>
<li>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</li>
<li>Default value:</li>
<li>Supports expression language: true</li>
</ul></li>
<li>Age Off Duration</strong>
<ul>
<li>Time interval to age off FlowFiles from the Distributed Cache Service; must specify time unit</li>
<li>Default value: none</li>
<li>Supports expression language: false</li>
</ul>
</li>
</ul>
<p>
<strong>Relationships:</strong>
</p>
<ul>
<li>duplicate
<ul>
<li>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.</li>
</ul>
</li>
<li>non-duplicate
<ul>
<li>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 <code>FlowFile
Description</code> property) will be stored in the cache.
</li>
</ul>
</li>
<li>failure
<ul>
<li>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</li>
</ul>
</li>
</ul>
<p>
<strong>See Also:</strong>
</p>
<ul>
<li><a href="../org.apache.nifi.distributed.cache.client.DistributedMapCacheClientService/index.html">Distributed Map Cache Client Service</a></li>
<li><a href="../org.apache.nifi.distributed.cache.server.map.DistributedMapCacheServer/index.html">Distributed Map Cache Server</a></li>
</ul>
</body>
</html>

View File

@ -34,58 +34,12 @@
FlowFiles in each iteration instead of 1. All other relationships will receive 1 FlowFile in each iteration. FlowFiles in each iteration instead of 1. All other relationships will receive 1 FlowFile in each iteration.
</p> </p>
<strong>Properties:</strong> <strong>Properties:</strong>
</p>
<p>In the list below, the names of required properties appear <p>In the list below, the names of required properties appear
in bold. Any other properties (not in bold) are considered optional. in bold. Any other properties (not in bold) are considered optional.
If a property has a default value, it is indicated. If a property If a property has a default value, it is indicated. If a property
supports the use of the NiFi Expression Language (or simply, supports the use of the NiFi Expression Language (or simply,
"expression language"), that is also indicated.</p> "expression language"), that is also indicated.</p>
<ul>
<li><strong>Number of Relationships</strong>
<ul>
<li>The number of relationships to which the load should be
distributed. A valid value is a non-negative integer.</li>
<li>Default value: 1</li>
<li>Supports expression language: false</li>
</ul></li>
<li><strong>Distribution Strategy</strong>
<ul>
<li>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:
<ul>
<li>round robin</li>
<li>next available</li>
<li>load distribution service</li>
</ul>
</li>
<li>Default value: round robin</li>
<li>Supports expression language: false</li>
</ul>
</li>
<li>Hostnames - this is required when using 'load distribution service' strategy
<ul>
<li>List of remote servers to distribute across. Each server must be FQDN and use either ',', ';', or 'space' as a delimiter</li>
<li>Default Value: none</li>
<li>Supports expression language: false</li>
</ul>
</li>
<li>Load Distribution Service ID - this is required when using 'load distribution service' strategy
<ul>
<li>The identifier of the Load Distribution Service</li>
<li>Default Value: none</li>
<li>Supports expression language: false</li>
</ul>
</li>
</ul>
<p> <p>
<strong>Relationships:</strong> <strong>Relationships:</strong>
</p> </p>

View File

@ -1,97 +0,0 @@
<!DOCTYPE html>
<html lang="en">
<!--
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.
-->
<head>
<meta charset="utf-8" />
<title>EncryptContent</title>
<link rel="stylesheet" href="../../css/component-usage.css" type="text/css" />
</head>
<body>
<!-- Processor Documentation ================================================== -->
<h2>Description:</h2>
<p>
This processor encrypts or decrypts FlowFiles.
<p>
<strong>Properties:</strong>
</p>
<p>
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.
</p>
<ul>
<li><strong>Mode</strong>
<ul>
<li>This property specifies whether the content should be encrypted or decrypted. Valid values are:
<ul>
<li>Encrypt</li>
<li>Decrypt</li>
</ul></li>
<li>Default value: Encrypt</li>
<li>Supports expression language: false</li>
</ul></li>
<li><strong>Encryption Algorithm</strong>
<ul>
<li>The encryption algorithm to use. Valid values are available in the property's drop down menu; they include:
<ul>
<li>MD5_128AES - requires only 'strong encryption policies'</li>
<li>MD5_256AES - requires only 'strong encryption policies'</li>
<li>SHA1_RC2 - requires only 'strong encryption policies'</li>
<li>SHA1_DES - requires only 'strong encryption policies'</li>
<li>MD5_192AES - requires only 'strong encryption policies'</li>
<li>MD5_DES - requires only 'strong encryption policies'</li>
<li>MD5_RC2 - requires only 'strong encryption policies'</li>
<li>SHA_192AES - requires unlimited JCE policies</li>
<li>SHA_40RC4 - requires unlimited JCE policies</li>
<li>SHA256_128AES - requires unlimited JCE policies</li>
<li>SHA_128RC2 - requires unlimited JCE policies</li>
<li>SHA_128AES - requires unlimited JCE policies</li>
<li>SHA256_192AES - requires unlimited JCE policies</li>
<li>SHA_2KEYTRIPLEDES - requires unlimited JCE policies</li>
<li>SHA256_256AES - requires unlimited JCE policies</li>
<li>SHA_40RC2 - requires unlimited JCE policies</li>
<li>SHA_256AES - requires unlimited JCE policies</li>
<li>SHA_3KEYTRIPLEDES - requires unlimited JCE policies</li>
<li>SHA_TWOFISH - requires unlimited JCE policies</li>
<li>SHA_128RC4 - requires unlimited JCE policies</li>
</ul></li>
<li>Default value: MD5_256AES</li>
<li>Supports expression language: false</li>
</ul></li>
<li><strong>Password</strong>
<ul>
<li>The password to use for encrypting or decrypting the data. Passwords longer than 16 characters require unlimited JCE policies</li>
<li>Default value: no default</li>
<li>Supports expression language: false</li>
</ul></li>
</ul>
<p>
<strong>Relationships:</strong>
</p>
<ul>
<li>failure
<ul>
<li>If a FlowFile cannot be encrypted or decrypted, then it follows this relationship</li>
</ul></li>
<li>success
<ul>
<li>If a FlowFile is successfully encrypted or decrypted, then it follows this relationship</li>
</ul></li>
</ul>
</body>
</html>

View File

@ -1,160 +0,0 @@
<!DOCTYPE html>
<html lang="en">
<!--
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.
-->
<head>
<meta charset="utf-8" />
<title>EvaluateRegularExpression</title>
<link rel="stylesheet" href="../../css/component-usage.css" type="text/css" />
</head>
<body>
<!-- Processor Documentation ================================================== -->
<h2>Description:</h2>
<p>
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.
<p>
<strong>Properties:</strong>
</p>
<p>
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.
</p>
<p>
<strong>Modifies Attributes:</strong>
</p>
<p>
This processor adds user-defined attributes.
</p>
<ul>
<li><strong>Character Set</strong>
<ul>
<li>The character set in which the file is encoded.</li>
<li>Default value: UTF-8</li>
<li>Supports expression language: false</li>
</ul></li>
<li><strong>Maximum Buffer Size</strong>
<ul>
<li>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.</li>
<li>Default value: 1MB</li>
<li>Supports expression language: false</li>
</ul></li>
<li><strong>Enable Canonical Equivalence</strong>
<ul>
<li>Indicates that two characters match only when their full canonical decompositions match.</li>
<li>Default value: false</li>
<li>Supports expression language: false</li>
</ul></li>
<li><strong>Enable Case-insensitive Matching</strong>
<ul>
<li>Indicates that two characters match even if they are in a different case. Can also be specified via the embeded flag (?i).</li>
<li>Default value: false</li>
<li>Supports expression language: false</li>
</ul></li>
<li><strong>Permit Whitespace and Comments in Pattern</strong>
<ul>
<li>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).</li>
<li>Default value: false</li>
<li>Supports expression language: false</li>
</ul></li>
<li><strong>Enable DOTALL Mode</strong>
<ul>
<li>Indicates that the expression '.' should match any character, including a line terminator. Can also be specified via the embeded flag (?s).</li>
<li>Default value: false</li>
<li>Supports expression language: false</li>
</ul></li>
<li><strong>Enable Literal Parsing of the Pattern</strong>
<ul>
<li>Indicates that Metacharacters and escape characters should be given no special meaning.</li>
<li>Default value: false</li>
<li>Supports expression language: false</li>
</ul></li>
<li><strong>Enable Multiline Mode</strong>
<ul>
<li>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).</li>
<li>Default value: false</li>
<li>Supports expression language: false</li>
</ul></li>
<li><strong>Enable Unicode-aware Case Folding</strong>
<ul>
<li>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).</li>
<li>Default value: false</li>
<li>Supports expression language: false</li>
</ul></li>
<li><strong>Enable Unicode Predefined Character Classes</strong>
<ul>
<li>Specifies conformance with the Unicode Technical Standard #18: Unicode Regular Expression Annex C: Compatibility Properties. Can also be specified via the embeded flag (?U).</li>
<li>Default value: false</li>
<li>Supports expression language: false</li>
</ul></li>
<li><strong>Enable Unix Lines Mode</strong>
<ul>
<li>Indicates that only the '\n' line terminator is recognized int the behavior of '.', '^', and '$'. Can also be specified via the embeded flag (?d).</li>
<li>Default value: false</li>
<li>Supports expression language: false</li>
</ul></li>
<li>
user-defined properties
<ul>
<li>The name of the attribute to put the Regular Expression result into.</li>
<li>Supports expression language: false</li>
</ul>
</li>
</ul>
<p>
<strong>Relationships:</strong>
</p>
<ul>
<li>
matched
<ul>
<li>FlowFiles are routed to this relationship when the Regular Expression is successfully evaluated and the FlowFile is modified as a result.</li>
</ul>
</li>
<li>
unmatched
<ul>
<li>FlowFiles are routed to this relationship when no provided Regular Expression matches the content of the FlowFile.</li>
</ul>
</li>
</ul>
</p>
</body>
</html>

View File

@ -0,0 +1,34 @@
<!DOCTYPE html>
<html lang="en">
<!--
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.
-->
<head>
<meta charset="utf-8" />
<title>EvaluateXPath</title>
<link rel="stylesheet" href="../../css/component-usage.css" type="text/css" />
</head>
<body>
<!-- Processor Documentation ================================================== -->
<p>
<strong>Modifies Attributes:</strong>
</p>
<p>
This processor adds user-defined attributes if the &lt;Destination&gt; property is set to
<code>flowfile-attribute</code>.
</p>
</body>
</html>

View File

@ -1,135 +0,0 @@
<!DOCTYPE html>
<html lang="en">
<!--
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.
-->
<head>
<meta charset="utf-8" />
<title>EvaluateXPath</title>
<link rel="stylesheet" href="../../css/component-usage.css" type="text/css" />
</head>
<body>
<!-- Processor Documentation ================================================== -->
<h2>Description:</h2>
<p>
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.
<p>
<strong>Properties:</strong>
</p>
<p>
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.
</p>
<p>
<strong>Modifies Attributes:</strong>
</p>
<p>
This processor adds user-defined attributes if the &lt;Destination&gt; property is set to
<code>flowfile-attribute</code>.
</p>
<ul>
<li>
<strong>Destination</strong>
<ul>
<li>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.</li>
<li>
Valid values are:
<ul>
<li>flowfile-content</li>
<li>flowfile-attribute</li>
</ul>
</li>
<li>Default value: flowfile-content</li>
<li>Supports expression language: false</li>
</ul>
</li>
<li>
<strong>Return Type</strong>
<ul>
<li>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'.")</li>
<li>
Valid values are:
<ul>
<li>auto-detect</li>
<li>nodeset</li>
<li>string</li>
</ul>
</li>
<li>Default value: auto-detect</li>
<li>Supports expression language: false</li>
</ul>
</li>
<li>
user-defined properties
<ul>
<li>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.</li>
<li>Supports expression language: false</li>
</ul>
</li>
</ul>
<p>
<strong>Relationships:</strong>
</p>
<ul>
<li>
failure
<ul>
<li>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.</li>
</ul>
</li>
<li>
matched
<ul>
<li>If the XPath is successfully evaluated and the FlowFile is
modified as a result, then the FlowFile follows this
relationship.</li>
</ul>
</li>
<li>
unmatched
<ul>
<li>If the XPath does not match the content of the FlowFile, then
the FlowFile follows this relationship.</li>
</ul>
</li>
</ul>
</p>
</body>
</html>

View File

@ -0,0 +1,161 @@
<!DOCTYPE html>
<html lang="en">
<!--
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.
-->
<head>
<meta charset="utf-8" />
<title>EvaluateXQuery</title>
<link rel="stylesheet" href="../../css/component-usage.css"
type="text/css" />
</head>
<body>
<!-- Processor Documentation ================================================== -->
<p>
<strong>Modifies Attributes:</strong>
</p>
<p>
This processor adds user-defined attributes if the &lt;Destination&gt;
property is set to
<code>flowfile-attribute</code>
.
</p>
<p>
<strong>Examples:</strong>
</p>
<p>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:</p>
<pre>
<code>
&lt;?xml version="1.0" encoding="UTF-8"?&gt;
&lt;?xml-stylesheet type="text/xsl" href="foo.xsl"?&gt;
&lt;ns:fruitbasket xmlns:ns="http://namespace/1"&gt;
&lt;fruit taste="crisp"&gt;
&lt;!-- Apples are my favorite--&gt;
&lt;name&gt;apple&lt;/name&gt;
&lt;color&gt;red&lt;/color&gt;
&lt;/fruit&gt;
&lt;fruit&gt;
&lt;name&gt;apple&lt;/name&gt;
&lt;color&gt;green&lt;/color&gt;
&lt;/fruit&gt;
&lt;fruit&gt;
&lt;name&gt;banana&lt;/name&gt;
&lt;color&gt;yellow&lt;/color&gt;
&lt;/fruit&gt;
&lt;fruit taste="sweet"&gt;
&lt;name&gt;orange&lt;/name&gt;
&lt;color&gt;orange&lt;/color&gt;
&lt;/fruit&gt;
&lt;fruit&gt;
&lt;name&gt;blueberry&lt;/name&gt;
&lt;color&gt;blue&lt;/color&gt;
&lt;/fruit&gt;
&lt;fruit taste="tart"&gt;
&lt;name&gt;raspberry&lt;/name&gt;
&lt;color&gt;red&lt;/color&gt;
&lt;/fruit&gt;
&lt;fruit&gt;
&lt;name&gt;none&lt;/name&gt;
&lt;color/&gt;
&lt;/fruit&gt;
&lt;/ns:fruitbasket&gt;
</code>
</pre>
<p>
<ul>
<li>XQuery to return all "fruit" nodes individually (7 Results):
<ul>
<li>//fruit</li>
</ul>
</li>
<li>XQuery to return only the first "fruit" node (1 Result):
<ul>
<li>//fruit[1]</li>
</ul>
</li>
<li>XQuery to return only the last "fruit" node (1 Result):
<ul>
<li>//fruit[count(//fruit)]</li>
</ul>
</li>
<li>XQuery to return all "fruit" nodes, wrapped in a "basket" tag
(1 Result):
<ul>
<li>&lt;basket&gt;{//fruit}&lt;/basket&gt;</li>
</ul>
</li>
<li>XQuery to return all "fruit" names individually (7 Results):
<ul>
<li>//fruit/text()</li>
</ul>
</li>
<li>XQuery to return only the first "fruit" name (1 Result):
<ul>
<li>//fruit[1]/text()</li>
</ul>
</li>
<li>XQuery to return only the last "fruit" name (1 Result):
<ul>
<li>//fruit[count(//fruit)]/text()</li>
</ul>
</li>
<li>XQuery to return all "fruit" names as a comma separated list
(1 Result):
<ul>
<li>string-join((for $x in //fruit return $x/name/text()), ',
')</li>
</ul>
</li>
<li>XQuery to return all "fruit" colors and names as a comma
separated list (1 Result):
<ul>
<li>string-join((for $y in (for $x in //fruit return
string-join(($x/color/text() , $x/name/text()), ' ')) return $y),
', ')</li>
</ul>
</li>
<li>XQuery to return all "fruit" colors and names as a comma
separated list (1 Result):
<ul>
<li>string-join((for $y in (for $x in //fruit return
string-join(($x/color/text() , $x/name/text()), ' ')) return $y),
', ')</li>
</ul>
</li>
<li>XQuery to return all "fruit" colors and names as a new line
separated list (1 Result):
<ul>
<li>string-join((for $y in (for $x in //fruit return
string-join(($x/color/text() , $x/name/text()), ' ')) return $y),
'\n')</li>
</ul>
</li>
</ul>
</body>
</html>

View File

@ -1,311 +0,0 @@
<!DOCTYPE html>
<html lang="en">
<!--
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.
-->
<head>
<meta charset="utf-8"/>
<title>EvaluateXQuery</title>
<link rel="stylesheet" href="../../css/component-usage.css" type="text/css"/>
</head>
<body>
<!-- Processor Documentation ================================================== -->
<h2>Description:</h2>
<p>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. </p>
<p> <strong>Properties:</strong> </p>
<p> 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. </p>
<p> <strong>Modifies Attributes:</strong> </p>
<p> This processor adds user-defined attributes if the
&lt;Destination&gt; property is set to <code>flowfile-attribute</code>
. </p>
<ul>
<li> <strong>Destination</strong>
<ul>
<li>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. </li>
<li> Valid values are:
<ul>
<li>flowfile-content</li>
<li>flowfile-attribute</li>
</ul>
</li>
<li>Default value: flowfile-content</li>
<li>Supports expression language: false</li>
</ul>
</li>
<li> <strong>Output: Method</strong>
<ul>
<li>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. </li>
<li> Valid values are:
<ul>
<li>xml</li>
<li>html</li>
<li>text</li>
</ul>
</li>
<li>Default value: xml</li>
<li>Supports expression language: false</li>
</ul>
</li>
<li> <strong>Output: Omit XML Declaration</strong>
<ul>
<li>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. </li>
<li> Valid values are:
<ul>
<li>true</li>
<li>false</li>
</ul>
</li>
<li>Default value: false</li>
<li>Supports expression language: false</li>
</ul>
</li>
<li> <strong>Output: Indent</strong>
<ul>
<li>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. </li>
<li> Valid values are:
<ul>
<li>true</li>
<li>false</li>
</ul>
</li>
<li>Default value: false</li>
<li>Supports expression language: false</li>
</ul>
</li>
<li> user-defined properties
<ul>
<li>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. </li>
<li>Supports expression language: false</li>
</ul>
</li>
</ul>
<p> <strong>Relationships:</strong> </p>
<ul>
<li> failure
<ul>
<li>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. </li>
</ul>
</li>
<li> matched
<ul>
<li>If the XQuery is successfully evaluated and the
FlowFile is modified as a result, then the FlowFile follows this
relationship. </li>
</ul>
</li>
<li> unmatched
<ul>
<li>If the XQuery does not match the content of the
FlowFile, then the FlowFile follows this relationship. </li>
</ul>
</li>
</ul>
<p> <strong>Examples:</strong> </p>
<p> 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:</p>
<pre><code>
&lt;?xml version="1.0" encoding="UTF-8"?&gt;
&lt;?xml-stylesheet type="text/xsl" href="foo.xsl"?&gt;
&lt;ns:fruitbasket xmlns:ns="http://namespace/1"&gt;
&lt;fruit taste="crisp"&gt;
&lt;!-- Apples are my favorite--&gt;
&lt;name&gt;apple&lt;/name&gt;
&lt;color&gt;red&lt;/color&gt;
&lt;/fruit&gt;
&lt;fruit&gt;
&lt;name&gt;apple&lt;/name&gt;
&lt;color&gt;green&lt;/color&gt;
&lt;/fruit&gt;
&lt;fruit&gt;
&lt;name&gt;banana&lt;/name&gt;
&lt;color&gt;yellow&lt;/color&gt;
&lt;/fruit&gt;
&lt;fruit taste="sweet"&gt;
&lt;name&gt;orange&lt;/name&gt;
&lt;color&gt;orange&lt;/color&gt;
&lt;/fruit&gt;
&lt;fruit&gt;
&lt;name&gt;blueberry&lt;/name&gt;
&lt;color&gt;blue&lt;/color&gt;
&lt;/fruit&gt;
&lt;fruit taste="tart"&gt;
&lt;name&gt;raspberry&lt;/name&gt;
&lt;color&gt;red&lt;/color&gt;
&lt;/fruit&gt;
&lt;fruit&gt;
&lt;name&gt;none&lt;/name&gt;
&lt;color/&gt;
&lt;/fruit&gt;
&lt;/ns:fruitbasket&gt;
</code></pre>
<p>
<ul>
<li>
XQuery to return all "fruit" nodes individually (7 Results):
<ul><li>//fruit</li></ul>
</li>
<li>
XQuery to return only the first "fruit" node (1 Result):
<ul><li>//fruit[1]</li></ul>
</li>
<li>
XQuery to return only the last "fruit" node (1 Result):
<ul><li>//fruit[count(//fruit)]</li></ul>
</li>
<li>
XQuery to return all "fruit" nodes, wrapped in a "basket" tag (1 Result):
<ul><li>&lt;basket&gt;{//fruit}&lt;/basket&gt;</li></ul>
</li>
<li>
XQuery to return all "fruit" names individually (7 Results):
<ul><li>//fruit/text()</li></ul>
</li>
<li>
XQuery to return only the first "fruit" name (1 Result):
<ul><li>//fruit[1]/text()</li></ul>
</li>
<li>
XQuery to return only the last "fruit" name (1 Result):
<ul><li>//fruit[count(//fruit)]/text()</li></ul>
</li>
<li>
XQuery to return all "fruit" names as a comma separated list (1 Result):
<ul><li>string-join((for $x in //fruit return $x/name/text()), ', ')</li></ul>
</li>
<li>
XQuery to return all "fruit" colors and names as a comma separated list (1 Result):
<ul><li>string-join((for $y in (for $x in //fruit return string-join(($x/color/text() , $x/name/text()), ' ')) return $y), ', ')</li></ul>
</li>
<li>
XQuery to return all "fruit" colors and names as a comma separated list (1 Result):
<ul><li>string-join((for $y in (for $x in //fruit return string-join(($x/color/text() , $x/name/text()), ' ')) return $y), ', ')</li></ul>
</li>
<li>
XQuery to return all "fruit" colors and names as a new line separated list (1 Result):
<ul><li>string-join((for $y in (for $x in //fruit return string-join(($x/color/text() , $x/name/text()), ' ')) return $y), '\n')</li></ul>
</li>
</ul>
</p>
</body>
</html>

View File

@ -0,0 +1,57 @@
<!DOCTYPE html>
<html lang="en">
<!--
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.
-->
<head>
<meta charset="utf-8" />
<title>ExecuteStreamCommand</title>
<link rel="stylesheet" href="../../css/component-usage.css" type="text/css" />
</head>
<body>
<!-- Processor Documentation ================================================== -->
<p>
<strong>Creates Attributes:</strong>
</p>
<table>
<thead>
<tr>
<th>Attribute Name</th>
<th>Description</th>
</tr>
</thead>
<tbody>
<tr>
<td>execution.command</td>
<td>The name of the command executed to create the new FlowFile</td>
</tr>
<tr>
<td>execution.command.args</td>
<td>The semi-colon delimited list of arguments</td>
</tr>
<tr>
<td>execution.status</td>
<td>The exit status code returned from executing the command</td>
</tr>
<tr>
<td>execution.error</td>
<td>Any error messages returned from executing the command</td>
</tr>
</tbody>
</table>
</body>
</html>

View File

@ -1,111 +0,0 @@
<!DOCTYPE html>
<html lang="en">
<!--
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.
-->
<head>
<meta charset="utf-8" />
<title>ExecuteStreamCommand</title>
<link rel="stylesheet" href="../../css/component-usage.css" type="text/css" />
</head>
<body>
<!-- Processor Documentation ================================================== -->
<h2> Description:</h2>
<p>This processor executes an external command on the contents of a FlowFile, and creates a new FlowFile with the
results of the command.</p>
<p>
<strong>Creates Attributes:</strong>
</p>
<table border="1">
<thead>
<tr>
<th>Attribute Name</th>
<th>Description</th>
</tr>
</thead>
<tbody>
<tr>
<td>execution.command</td>
<td>The name of the command executed to create the new FlowFile</td>
</tr>
<tr>
<td>execution.command.args</td>
<td>The semi-colon delimited list of arguments</td>
</tr>
<tr>
<td>execution.status</td>
<td>The exit status code returned from executing the command</td>
</tr>
<tr>
<td>execution.error</td>
<td>Any error messages returned from executing the command</td>
</tr>
</tbody>
</table>
<p>
<strong>Properties:</strong>
</p>
<p>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.</p>
<ul>
<li><strong>Command Path</strong>
<ul>
<li>Specifies the command to be executed; if just the name of an executable is provided, it must be in the user's
environment PATH.</li>
<li>Default value: none</li>
<li>Supports expression language: true</li>
</ul>
</li>
<li>Command Arguments
<ul>
<li>The arguments to supply to the executable delimited by the ';' character. Each argument may be an Expression Language
statement.</li>
<li>Default value: none</li>
<li>Supports expression language: true</li>
</ul>
</li>
<li>Working Directory
<ul>
<li>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.</li>
<li>Default value: none (which means whatever NiFi's current working directory is...probably the root of the NiFi
installation directory.)</li>
<li>Supports expression language: true</li>
</ul>
</li>
</ul>
<p>
<strong>Relationships:</strong>
</p>
<ul>
<li>original
<ul>
<li>The destination path for the original incoming FlowFile. This FlowFile will have the created attributes listed above.</li>
</ul>
</li>
<li>output-stream
<ul>
<li>The destination path for the FlowFile created from the command's output. This FlowFile will have the created attributes listed above.</li>
</ul>
</li>
</ul>
</body>
</html>

View File

@ -1,64 +0,0 @@
<!DOCTYPE html>
<html lang="en">
<!--
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.
-->
<head>
<meta charset="utf-8" />
<title>GenerateFlowFile</title>
<link rel="stylesheet" href="../../css/component-usage.css" type="text/css" />
</head>
<body>
<!-- Processor Documentation ================================================== -->
<h2>Description:</h2>
<p>
This processor creates FlowFiles of random data to be used for load testing purposes.
</p>
<p>
<strong>Properties:</strong>
</p>
<p>
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.
</p>
<ul>
<li><strong>File Size</strong>
<ul>
<li>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.</li>
<li>Default value: no default</li>
<li>Supports expression language: false</li>
</ul></li>
<li><strong>Batch Size</strong>
<ul>
<li>The number of FlowFiles to be transferred in each invocation. The value must be a non-negative integer.</li>
<li>Default value: 1</li>
<li>Supports expression language: false</li>
</ul></li>
</ul>
<p>
<strong>Relationships:</strong>
</p>
<ul>
<li>success
<ul>
<li>If FlowFiles are successfully generated, they follow this relationship.</li>
</ul></li>
</ul>
</body>
</html>

View File

@ -0,0 +1,74 @@
<!DOCTYPE html>
<html lang="en">
<!--
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.
-->
<head>
<meta charset="utf-8" />
<title>GetFTP</title>
<link rel="stylesheet" href="../../css/component-usage.css" type="text/css" />
</head>
<body>
<!-- Processor Documentation ================================================== -->
<p>
<strong>Modifies Attributes:</strong>
</p>
<table>
<thead>
<tr>
<th>Attribute Name</th>
<th>Description</th>
</tr>
</thead>
<tbody>
<tr>
<td>filename</td>
<td>The filename is set to the name of the file on the remote server</td>
</tr>
<tr>
<td>path</td>
<td>The path is set to the path of the file's
directory on the remote server. For example, if the &lt;Remote Path&gt;
property is set to <code>/tmp</code>, files picked up from /tmp
will have the path attribute set to <code>/tmp</code>. If the
&lt;Search Recursively&gt; property is set to <code>true</code>
and a file is picked up from /tmp/abc/1/2/3, then the path
attribute will be set to <code>/tmp/abc/1/2/3</code></td>
</tr>
<tr>
<td>file.lastModifiedTime</td>
<td>The date and time that the source file was last modified.</td>
</tr>
<tr>
<td>file.owner</td>
<td>The numeric owner id of the source file.</td>
</tr>
<tr>
<td>file.group</td>
<td>The numeric group id of the source file.</td>
</tr>
<tr>
<td>file.permissions</td>
<td>The read/write/execute permissions of the source file.</td>
</tr>
<tr>
<td>absolute.path</td>
<td>The full/absolute path from where a file was picked up. The current 'path' attribute is still populated, but may be a relative path.</td>
</tr>
</tbody>
</table>
</body>
</html>

View File

@ -1,227 +0,0 @@
<!DOCTYPE html>
<html lang="en">
<!--
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.
-->
<head>
<meta charset="utf-8" />
<title>GetFTP</title>
<link rel="stylesheet" href="../../css/component-usage.css" type="text/css" />
</head>
<body>
<!-- Processor Documentation ================================================== -->
<h2>Description:</h2>
<p>This processor fetches files from an FTP server and creates
FlowFiles from them.</p>
<p>
<strong>Modifies Attributes:</strong>
</p>
<table border="1">
<thead>
<tr>
<th>Attribute Name</th>
<th>Description</th>
</tr>
</thead>
<tbody>
<tr>
<td>filename</td>
<td>The filename is set to the name of the file on the remote server</td>
</tr>
<tr>
<td>path</td>
<td>The path is set to the path of the file's
directory on the remote server. For example, if the &lt;Remote Path&gt;
property is set to <code>/tmp</code>, files picked up from /tmp
will have the path attribute set to <code>/tmp</code>. If the
&lt;Search Recursively&gt; property is set to <code>true</code>
and a file is picked up from /tmp/abc/1/2/3, then the path
attribute will be set to <code>/tmp/abc/1/2/3</code></td>
</tr>
<tr>
<td>file.lastModifiedTime</td>
<td>The date and time that the source file was last modified.</td>
</tr>
<tr>
<td>file.owner</td>
<td>The numeric owner id of the source file.</td>
</tr>
<tr>
<td>file.group</td>
<td>The numeric group id of the source file.</td>
</tr>
<tr>
<td>file.permissions</td>
<td>The read/write/execute permissions of the source file.</td>
</tr>
<tr>
<td>absolute.path</td>
<td>The full/absolute path from where a file was picked up. The current 'path' attribute is still populated, but may be a relative path.</td>
</tr>
</tbody>
</table>
<p>
<strong>Properties:</strong>
</p>
<p>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.</p>
<ul>
<li><strong>Hostname</strong>
<ul>
<li>The fully qualified hostname or IP address of the remote
system.</li>
<li>Default value: no default</li>
<li>Supports expression language: true</li>
</ul></li>
<li><strong>Port</strong>
<ul>
<li>The port that the remote system is listening on for file transfers.</li>
<li>Default value: 21</li>
<li>Supports expression language: false</li>
</ul></li>
<li><strong>Username</strong>
<ul>
<li>The username for the user account.</li>
<li>Default value: no default</li>
<li>Supports expression language: false</li>
</ul></li>
<li>Password
<ul>
<li>The password for the user account.</li>
<li>Default value: no default</li>
<li>Supports expression language: false</li>
</ul>
</li>
<li>Connection Mode
<ul>
<li>The FTP connection mode. Valid options are: Active or Passive. For most clients, this should be set to Passive.</li>
<li>Default value: Passive</li>
<li>Supports expression language: false</li>
</ul></li>
<li>Transfer Mode
<ul>
<li>The FTP transfer mode. Valid options are: Binary or ASCII.</li>
<li>Default value: Binary</li>
<li>Supports expression language: false</li>
</ul></li>
<li>Remote Path
<ul>
<li>The path on the remote system from which to pull files. If
not specified, the user's home directory will be used.</li>
<li>Default value: no default</li>
<li>Supports expression language: true</li>
</ul>
</li>
<li>File Filter Regex
<ul>
<li>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.</li>
<li>Default value: no default</li>
<li>Supports expression language: false</li>
</ul>
</li>
<li><strong>Polling Interval</strong>
<ul>
<li>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.</li>
<li>Default value: 60 sec</li>
<li>Supports expression language: false</li>
</ul></li>
<li><strong>Search Recursively</strong>
<ul>
<li>A Boolean value (true/false), indicating whether to
traverse subdirectories and pull files from them.</li>
<li>Default value: false</li>
<li>Supports expression language: false</li>
</ul></li>
<li><strong>Ignore Dotted Files</strong>
<ul>
<li>A Boolean value (true/false), indicating whether to ignore
files whose names begin with a dot (.), and, therefore, not pick
them up.</li>
<li>Default value: true</li>
<li>Supports expression language: false</li>
</ul></li>
<li><strong>Delete Original</strong>
<ul>
<li>A Boolean value (true/false), indicating whether to delete
the original copy of the file on the remote system after it has
been transferred.</li>
<li>Default value: true</li>
<li>Supports expression language: false</li>
</ul></li>
<li><strong>Connection Timeout</strong>
<ul>
<li>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.</li>
<li>Default value: 30 sec</li>
<li>Supports expression language: false</li>
</ul></li>
<li><strong>Data Timeout</strong>
<ul>
<li>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.</li>
<li>Default value: 30 sec</li>
<li>Supports expression language: false</li>
</ul></li>
<li><strong>Max Selects</strong>
<ul>
<li>The maximum number of files to pull in a single connection.
</li>
<li>Default value: 100</li>
<li>Supports expression language: false</li>
</ul></li>
<li><strong>Remote Poll Batch Size</strong>
<ul>
<li>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.</li>
<li>Default value: 5000</li>
<li>Supports expression language: false</li>
</ul></li>
<li><strong>Use Natural Ordering</strong>
<ul>
<li>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.</li>
<li>Default value: false</li>
<li>Supports expression language: false</li>
</ul></li>
</ul>
<p>
<strong>Relationships:</strong>
</p>
<ul>
<li>success
<ul>
<li>If FlowFiles are successfully retrieved by FTP, they follow
this relationship.</li>
</ul>
</li>
</ul>
</body>
</html>

View File

@ -0,0 +1,79 @@
<!DOCTYPE html>
<html lang="en">
<!--
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.
-->
<head>
<meta charset="utf-8" />
<title>GetFile</title>
<link rel="stylesheet" href="../../css/component-usage.css" type="text/css" />
</head>
<body>
<!-- Processor Documentation ================================================== -->
<p>
<strong>Modifies Attributes:</strong>
</p>
<table>
<thead>
<tr>
<th>Attribute Name</th>
<th>Description</th>
</tr>
</thead>
<tbody>
<tr>
<td>filename</td>
<td>The filename is set to the name of the file on disk</td>
</tr>
<tr>
<td>path</td>
<td>The path is set to the relative path of the file's directory on disk. For example, if the &lt;Input Directory&gt;
property is set to <code>/tmp</code>, files picked up from /tmp will have the path attribute set to <code>./</code>.
If the &lt;Recurse Subdirectories&gt; property is set to <code>true</code> and a file is picked up
from /tmp/abc/1/2/3, then the path attribute will be set to <code>abc/1/2/3</code>
</tr>
<tr>
<td>file.creationTime</td>
<td>The date and time that the file was created. May not work on all file systems</td>
</tr>
<tr>
<td>file.lastModifiedTime</td>
<td>The date and time that the file was last modified. May not work on all file systems</td>
</tr>
<tr>
<td>file.lastAccessTime</td>
<td>The date and time that the file was last accessed. May not work on all file systems</td>
</tr>
<tr>
<td>file.owner</td>
<td>The owner of the file. May not work on all file systems</td>
</tr>
<tr>
<td>file.group</td>
<td>The group owner of the file. May not work on all file systems</td>
</tr>
<tr>
<td>file.permissions</td>
<td>The read/write/execute permissions of the file. May not work on all file systems</td>
</tr>
<tr>
<td>absolute.path</td>
<td>The full/absolute path from where a file was picked up. The current 'path' attribute is still populated, but may be a relative path.</td>
</tr>
</tbody>
</table>
</body>
</html>

View File

@ -1,186 +0,0 @@
<!DOCTYPE html>
<html lang="en">
<!--
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.
-->
<head>
<meta charset="utf-8" />
<title>GetFile</title>
<link rel="stylesheet" href="../../css/component-usage.css" type="text/css" />
</head>
<body>
<!-- Processor Documentation ================================================== -->
<h2>Description:</h2>
<p>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.</p>
<p>
<strong>Modifies Attributes:</strong>
</p>
<table border="1">
<thead>
<tr>
<th>Attribute Name</th>
<th>Description</th>
</tr>
</thead>
<tbody>
<tr>
<td>filename</td>
<td>The filename is set to the name of the file on disk</td>
</tr>
<tr>
<td>path</td>
<td>The path is set to the relative path of the file's directory on disk. For example, if the &lt;Input Directory&gt;
property is set to <code>/tmp</code>, files picked up from /tmp will have the path attribute set to <code>./</code>.
If the &lt;Recurse Subdirectories&gt; property is set to <code>true</code> and a file is picked up
from /tmp/abc/1/2/3, then the path attribute will be set to <code>abc/1/2/3</code>
</tr>
<tr>
<td>file.creationTime</td>
<td>The date and time that the file was created. May not work on all file systems</td>
</tr>
<tr>
<td>file.lastModifiedTime</td>
<td>The date and time that the file was last modified. May not work on all file systems</td>
</tr>
<tr>
<td>file.lastAccessTime</td>
<td>The date and time that the file was last accessed. May not work on all file systems</td>
</tr>
<tr>
<td>file.owner</td>
<td>The owner of the file. May not work on all file systems</td>
</tr>
<tr>
<td>file.group</td>
<td>The group owner of the file. May not work on all file systems</td>
</tr>
<tr>
<td>file.permissions</td>
<td>The read/write/execute permissions of the file. May not work on all file systems</td>
</tr>
<tr>
<td>absolute.path</td>
<td>The full/absolute path from where a file was picked up. The current 'path' attribute is still populated, but may be a relative path.</td>
</tr>
</tbody>
</table>
<p>
<strong>Properties:</strong>
</p>
<p>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.</p>
<ul>
<li><strong>Input Directory</strong>
<ul>
<li>The input directory from which to pull files.</li>
<li>Default value: no default</li>
<li>Supports expression language: true</li>
</ul></li>
<li><strong>File Filter</strong>
<ul>
<li>Only files whose names match the given regular expression
are picked up.</li>
<li>Default value: [^\.].*</li>
<li>Supports expression language: false</li>
</ul></li>
<li><strong>Keep Source File</strong>
<ul>
<li>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.</li>
<li>Default value: false</li>
<li>Supports expression language: false</li>
</ul></li>
<li><strong>Recurse Subdirectories</strong>
<ul>
<li>A Boolean value (true/false), indicating whether to pull
files from subdirectories.</li>
<li>Default value: true</li>
<li>Supports expression language: false</li>
</ul></li>
<li><strong>Polling Interval</strong>
<ul>
<li>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. <b>NOTE:</b> If a directory
listing is empty, the processor will yield for the full yield duration and will not poll during that
time.
</li>
<li>Default value: 0 sec</li>
<li>Supports expression language: false</li>
</ul></li>
<li><strong>Ignore Hidden Files</strong>
<ul>
<li>A Boolean value (true/false), indicating whether to ignore
hidden files.</li>
<li>Default value: true</li>
<li>Supports expression language: false</li>
</ul></li>
<li><strong>Minimum File Age</strong>
<ul>
<li>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.</li>
<li>Default value: 0 sec</li>
<li>Supports expression language: false</li>
</ul></li>
<li>Maximum File Age
<ul>
<li>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.</li>
<li>Default value: no default</li>
<li>Supports expression language: false</li>
</ul>
</li>
<li><strong>Minimum File Size</strong>
<ul>
<li>The minimum size that a file must be in order to be pulled.</li>
<li>Default value: 0 B</li>
<li>Supports expression language: false</li>
</ul></li>
<li>Maximum File Size
<ul>
<li>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.</li>
<li>Default value: no default</li>
<li>Supports expression language: false</li>
</ul>
</li>
</ul>
<p>
<strong>Relationships:</strong>
</p>
<ul>
<li>success
<ul>
<li>If FlowFiles are successfully selected, they follow this
relationship.</li>
</ul>
</li>
</ul>
</body>
</html>

View File

@ -0,0 +1,45 @@
<!DOCTYPE html>
<html lang="en">
<!--
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.
-->
<head>
<meta charset="utf-8" />
<title>GetHTTP</title>
<link rel="stylesheet" href="../../css/component-usage.css" type="text/css" />
</head>
<body>
<!-- Processor Documentation ================================================== -->
<h2>Description:</h2>
<p>
<strong>Modifies Attributes:</strong>
</p>
<table>
<thead>
<tr>
<th>Attribute Name</th>
<th>Description</th>
</tr>
</thead>
<tbody>
<tr>
<td>filename</td>
<td>The filename is set to the name of the file on the remote server</td>
</tr>
</tbody>
</table>
</body>
</html>

View File

@ -1,143 +0,0 @@
<!DOCTYPE html>
<html lang="en">
<!--
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.
-->
<head>
<meta charset="utf-8" />
<title>GetHTTP</title>
<link rel="stylesheet" href="../../css/component-usage.css" type="text/css" />
</head>
<body>
<!-- Processor Documentation ================================================== -->
<h2>Description:</h2>
<p>This processor fetches files via HTTP and creates FlowFiles
from them.
</p>
<p>
<strong>Modifies Attributes:</strong>
</p>
<table border="1">
<thead>
<tr>
<th>Attribute Name</th>
<th>Description</th>
</tr>
</thead>
<tbody>
<tr>
<td>filename</td>
<td>The filename is set to the name of the file on the remote server</td>
</tr>
</tbody>
</table>
<p>
<strong>Properties:</strong>
</p>
<p>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.</p>
<ul>
<li><strong>URL</strong>
<ul>
<li>The URL from which to pull files</li>
<li>Default value: no default</li>
<li>Supports expression language: false</li>
</ul></li>
<li><strong>Filename</strong>
<ul>
<li>The filename to assign to the file when pulled.</li>
<li>Default value: no default</li>
<li>Supports expression language: false</li>
</ul></li>
<li>SSL Context Service
<ul>
<li>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.</li>
<li>Default value: no default</li>
<li>Supports expression language: false</li>
</ul>
</li>
<li>Username
<ul>
<li>The username required to access the URL.</li>
<li>Default value: no default</li>
<li>Supports expression language: false</li>
</ul>
</li>
<li>Password
<ul>
<li>The password required to access the URL.</li>
<li>Default value: no default</li>
<li>Supports expression language: false</li>
</ul>
</li>
<li><strong>Connection Timeout</strong>
<ul>
<li>The amount of time to wait before timing out while creating
a connection.</li>
<li>Default value: 30 sec</li>
<li>Supports expression language: false</li>
</ul></li>
<li><strong>Data Timeout</strong>
<ul>
<li>The amount of time to wait before timing out while
transferring data.</li>
<li>Default value: 30 sec</li>
<li>Supports expression language: false</li>
</ul></li>
<li>User Agent
<ul>
<li>What to report as the user agent when a connection is made
to the remote server.</li>
<li>Default value: no default</li>
<li>Supports expression language: false</li>
</ul>
</li>
<li>Accept Content-Type
<ul>
<li>If specified, requests will only accept the provided
Content-Type.</li>
<li>Default value: no default</li>
<li>Supports expression language: false</li>
</ul>
</li>
<li>Follow Redirects
<ul>
<li>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.</li>
<li>Default value: false</li>
<li>Supports expression language: false</li>
</ul>
</li>
</ul>
<p>
<strong>Relationships:</strong>
</p>
<ul>
<li>success
<ul>
<li>All incoming FlowFiles follow this relationship.</li>
</ul>
</li>
</ul>
</body>
</html>

View File

@ -1,118 +0,0 @@
<!DOCTYPE html>
<html lang="en">
<!--
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.
-->
<head>
<meta charset="utf-8" />
<title>GetJMSQueue</title>
<link rel="stylesheet" href="../../css/component-usage.css" type="text/css" />
</head>
<body>
<!-- Processor Documentation ================================================== -->
<h2>Description:</h2>
<p>
This processor pulls messages from a JMS Queue, creating a FlowFile for each JMS message or bundle of messages, as configured.
</p>
<p>
<strong>Properties:</strong>
</p>
<p>
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.
</p>
<ul>
<li><strong>JMS Provider</strong>
<ul>
<li>This property specifies the provider used for the JMS server. Available options include:
<ul>
<li>ActiveMQ</li>
</ul></li>
<li>Default value: ActiveMQ</li>
<li>Supports expression language: false</li>
</ul></li>
<li><strong>URL</strong>
<ul>
<li>The URL of the JMS server.</li>
<li>Default value: no default</li>
<li>Supports expression language: false</li>
</ul></li>
<li><strong>Destination Name</strong>
<ul>
<li>The name of the JMS queue to use.</li>
<li>Default value: no default</li>
<li>Supports expression language: false</li>
</ul></li>
<li><strong>Communications Timeout</strong>
<ul>
<li>The amount of time to wait when attempting to receive a message before giving up and assuming failure.</li>
<li>Default value: 30 sec</li>
<li>Supports expression language: false</li>
</ul></li>
<li><strong>Message Batch Size</strong>
<ul>
<li>The number of messages to pull in a single iteration of the processor.</li>
<li>Default value: 10</li>
<li>Supports expression language: false</li>
</ul></li>
<li>Username
<ul>
<li>The username used for authentication and authorization.</li>
<li>Default value: no default</li>
<li>Supports expression language: false</li>
</ul></li>
<li>Password
<ul>
<li>The password used for authentication and authorization.</li>
<li>Default value: no default</li>
<li>Supports expression language: false</li>
</ul></li>
<li><strong>Acknowledgement Mode</strong>
<ul>
<li>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.</li>
<li>Default value: Client Acknowledge</li>
<li>Supports expression language: false</li>
</ul></li>
<li>Message Selector
<ul>
<li>The JMS Message Selector to use in order to narrow the messages that are pulled.</li>
<li>Default value: no default</li>
<li>Supports expression language: false</li>
</ul></li>
<li><strong>Copy of JMS Properties Attributes</strong>
<ul>
<li>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. </li>
<li>Default value: true</li>
<li>Supports expression language: false</li>
</ul></li>
<li><strong>Client ID Prefix</strong>
<ul>
<li>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.</li>
<li>Default value: no default</li>
<li>Supports expression language: false</li>
</ul></li>
</ul>
<p>
<strong>Relationships:</strong>
</p>
<ul>
<li>success
<ul>
<li>All incoming FlowFiles follow this relationship.</li>
</ul></li>
</ul>
</body>
</html>

View File

@ -1,122 +0,0 @@
<!DOCTYPE html>
<html lang="en">
<!--
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.
-->
<head>
<meta charset="utf-8" />
<title>GetJMSTopic</title>
<link rel="stylesheet" href="../../css/component-usage.css" type="text/css" />
</head>
<body>
<!-- Processor Documentation ================================================== -->
<h2>Description:</h2>
<p>
This processor pulls messages from a JMS Topic, creating a FlowFile for each JMS message or bundle of messages, as configured.
</p>
<p>
<strong>Properties:</strong>
</p>
<p>
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.
</p>
<ul>
<li><strong>JMS Provider</strong>
<ul>
<li>This property specifies the provider used for the JMS server. Available options include:
<ul>
<li>ActiveMQ</li>
</ul></li>
<li>Supports expression language: false</li>
</ul></li>
<li><strong>URL</strong>
<ul>
<li>The URL of the JMS server.</li>
<li>Default value: no default</li>
<li>Supports expression language: false</li>
</ul></li>
<li><strong>Destination Name</strong>
<ul>
<li>The name of the JMS topic to use.</li>
<li>Default value: no default</li>
<li>Supports expression language: false</li>
</ul></li>
<li><strong>Communications Timeout</strong>
<ul>
<li>The amount of time to wait when attempting to receive a message before giving up and assuming failure.</li>
<li>Default value: 30 sec</li>
<li>Supports expression language: false</li>
</ul></li>
<li><strong>Message Batch Size</strong>
<ul>
<li>The number of messages to pull in a single iteration of the processor.</li>
<li>Default value: 10</li>
<li>Supports expression language: false</li>
</ul></li>
<li>Username
<ul>
<li>The username used for authentication and authorization.</li>
<li>Default value: no default</li>
<li>Supports expression language: false</li>
</ul></li>
<li>Password
<ul>
<li>The password used for authentication and authorization.</li>
<li>Default value: no default</li>
<li>Supports expression language: false</li>
</ul></li>
<li><strong>Acknowledgement Mode</strong>
<ul>
<li>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.</li>
<li>Default value: Client Acknowledge</li>
<li>Supports expression language: false</li>
</ul></li>
<li>Message Selector
<ul>
<li>The JMS Message Selector to use in order to narrow the messages that are pulled.</li>
<li>Default value: no default</li>
<li>Supports expression language: false</li>
</ul></li>
<li><strong>Copy of JMS Properties Attributes</strong>
<ul>
<li>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. </li>
<li>Default value: true</li>
<li>Supports expression language: false</li>
</ul></li>
<li><strong>Client ID Prefix</strong>
<ul>
<li>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.</li>
<li>Default value: no default</li>
<li>Supports expression language: false</li>
</ul></li>
<li><strong>Use Durable Subscription</strong>
<ul>
<li>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. </li>
<li>Default value: false</li>
<li>Supports expression language: false</li>
</ul></li>
</ul>
<p>
<strong>Relationships:</strong>
</p>
<ul>
<li>success
<ul>
<li>All incoming FlowFiles follow this relationship.</li>
</ul></li>
</ul>
</body>
</html>

View File

@ -0,0 +1,80 @@
<!DOCTYPE html>
<html lang="en">
<!--
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.
-->
<head>
<meta charset="utf-8" />
<title>GetSFTP</title>
<link rel="stylesheet" href="../../css/component-usage.css" type="text/css" />
</head>
<body>
<!-- Processor Documentation ================================================== -->
<p>
<strong>Modifies Attributes:</strong>
</p>
<table>
<thead>
<tr>
<th>Attribute Name</th>
<th>Description</th>
</tr>
</thead>
<tbody>
<tr>
<td>filename</td>
<td>The filename is set to the name of the file on the remote server</td>
</tr>
<tr>
<td>path</td>
<td>The path is set to the path of the file's
directory on the remote server. For example, if the &lt;Remote Path&gt;
property is set to <code>/tmp</code>, files picked up from /tmp
will have the path attribute set to <code>/tmp</code>. If the
&lt;Search Recursively&gt; property is set to <code>true</code>
and a file is picked up from /tmp/abc/1/2/3, then the path
attribute will be set to <code>/tmp/abc/1/2/3</code></td>
</tr>
<tr>
<td>file.lastModifiedTime</td>
<td>The date and time that the source file was last modified.</td>
</tr>
<tr>
<td>file.owner</td>
<td>The numeric owner id of the source file.</td>
</tr>
<tr>
<td>file.group</td>
<td>The numeric group id of the source file.</td>
</tr>
<tr>
<td>file.permissions</td>
<td>The read/write/execute permissions of the source file.</td>
</tr>
<tr>
<td>absolute.path</td>
<td>The full/absolute path from where a file was picked up. The current 'path' attribute is still populated, but may be a relative path.</td>
</tr>
</tbody>
</table>
<p>
<strong>See Also:</strong>
<ul>
<li><a href="../org.apache.nifi.processors.standard.PutSFTP/index.html">PutSFTP</a></li>
</ul>
</body>
</html>

View File

@ -1,250 +0,0 @@
<!DOCTYPE html>
<html lang="en">
<!--
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.
-->
<head>
<meta charset="utf-8" />
<title>GetSFTP</title>
<link rel="stylesheet" href="../../css/component-usage.css" type="text/css" />
</head>
<body>
<!-- Processor Documentation ================================================== -->
<h2>Description:</h2>
<p>This processor pulls files from an SFTP server and creates
FlowFiles to encapsulate them.
</p>
<p>
<strong>Modifies Attributes:</strong>
</p>
<table border="1">
<thead>
<tr>
<th>Attribute Name</th>
<th>Description</th>
</tr>
</thead>
<tbody>
<tr>
<td>filename</td>
<td>The filename is set to the name of the file on the remote server</td>
</tr>
<tr>
<td>path</td>
<td>The path is set to the path of the file's
directory on the remote server. For example, if the &lt;Remote Path&gt;
property is set to <code>/tmp</code>, files picked up from /tmp
will have the path attribute set to <code>/tmp</code>. If the
&lt;Search Recursively&gt; property is set to <code>true</code>
and a file is picked up from /tmp/abc/1/2/3, then the path
attribute will be set to <code>/tmp/abc/1/2/3</code></td>
</tr>
<tr>
<td>file.lastModifiedTime</td>
<td>The date and time that the source file was last modified.</td>
</tr>
<tr>
<td>file.owner</td>
<td>The numeric owner id of the source file.</td>
</tr>
<tr>
<td>file.group</td>
<td>The numeric group id of the source file.</td>
</tr>
<tr>
<td>file.permissions</td>
<td>The read/write/execute permissions of the source file.</td>
</tr>
<tr>
<td>absolute.path</td>
<td>The full/absolute path from where a file was picked up. The current 'path' attribute is still populated, but may be a relative path.</td>
</tr>
</tbody>
</table>
<p>
<strong>Properties:</strong>
</p>
<p>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.</p>
<ul>
<li><strong>Hostname</strong>
<ul>
<li>The fully qualified hostname or IP address of the remote
system.</li>
<li>Default value: no default</li>
<li>Supports expression language: true</li>
</ul></li>
<li><strong>Port</strong>
<ul>
<li>The port that the remote system is listening on for file transfers.</li>
<li>Default value: 22</li>
<li>Supports expression language: false</li>
</ul></li>
<li><strong>Username</strong>
<ul>
<li>The username for the user account on the remote system.</li>
<li>Default value: no default</li>
<li>Supports expression language: false</li>
</ul></li>
<li>Password
<ul>
<li>The password for the user account.</li>
<li>Default value: no default</li>
<li>Supports expression language: false</li>
</ul>
</li>
<li>Private Key Path
<ul>
<li>The fully qualified path for the private key file.</li>
<li>Default value: no default</li>
<li>Supports expression language: false</li>
</ul>
</li>
<li>Private Key Passphrase
<ul>
<li>The password for the private key.</li>
<li>Default value: no default</li>
<li>Supports expression language: false</li>
</ul>
</li>
<li>Remote Path
<ul>
<li>The path on the remote system from which to pull files. If
not specified, the user's home directory will be used.</li>
<li>Default value: no default</li>
<li>Supports expression language: true</li>
</ul>
</li>
<li>File Filter Regex
<ul>
<li>Provides a Java Regular Expression for filtering Filenames; if a filter is supplied, only files
whose names match that Regular Expression will be fetched</li>
<li>Default value: no default</li>
<li>Supports expression language: false</li>
</ul>
</li>
<li><strong>Polling Interval</strong>
<ul>
<li>Determines how long to wait between fetching the listing for new files</li>
<li>Default value: 60 secs</li>
<li>Supports expression language: false</li>
</ul>
</li>
<li><strong>Search Recursively</strong>
<ul>
<li>If true, will pull files from arbitrarily nested subdirectories; otherwise, will not traverse subdirectories</li>
<li>Default value: false</li>
<li>Supports expression language: false</li>
</ul>
</li>
<li><strong>Ignore Dotted Files</strong>
<ul>
<li>If true, files whose names begin with a dot (\".\") will be ignored</li>
<li>Default value: true</li>
<li>Supports expression language: false</li>
</ul>
</li>
<li><strong>Delete Original</strong>
<ul>
<li>Determines whether or not the file is deleted from the remote system after it has been successfully transferred</li>
<li>Default value: true</li>
<li>Supports expression language: false</li>
</ul>
</li>
<li><strong>Connection Timeout</strong>
<ul>
<li>The amount of time to wait before timing out while creating a connection.</li>
<li>Default value: 30 sec</li>
<li>Supports expression language: false</li>
</ul></li>
<li><strong>Data Timeout</strong>
<ul>
<li>The amount of time to wait before timing out while
transferring data.</li>
<li>Default value: 30 sec</li>
<li>Supports expression language: false</li>
</ul></li>
<li>Host Key File
<ul>
<li>The local file path to the host key file; if not supplied,
no host key file will be used.</li>
<li>Default value: no default</li>
<li>Supports expression language: false</li>
</ul>
</li>
<li><strong>Max Selects</strong>
<ul>
<li>The maximum number of files to pull in a single connection.</li>
<li>Default value: 100</li>
<li>Supports expression language: false</li>
</ul></li>
<li><strong>Remote Poll Batch Size</strong>
<ul>
<li>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.</li>
<li>Default value: 5000</li>
<li>Supports expression language: false</li>
</ul></li>
<li><strong>Strict Host Key Checking</strong>
<ul>
<li>A Boolean value (true/false), indicating whether to apply
strict enforcement of host keys.</li>
<li>Default value: false</li>
<li>Supports expression language: false</li>
</ul></li>
<li><strong>Use Compression</strong>
<ul>
<li>A Boolean value (true/false), indicating whether to use
ZLIB compression when transferring files.</li>
<li>Default value: false</li>
<li>Supports expression language: false</li>
</ul></li>
<li><strong>Use Natural Ordering</strong>
<ul>
<li>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</li>
<li>Default value: false</li>
<li>Supports expression language: false</li>
</ul></li>
</ul>
<p>
<strong>Relationships:</strong>
</p>
<ul>
<li>success
<ul>
<li>If FlowFiles are successfully transferred, then they follow
this relationship.</li>
</ul>
</li>
</ul>
<p>
<strong>See Also:</strong>
<ul>
<li><a href="../org.apache.nifi.processors.standard.PutSFTP/index.html">PutSFTP</a></li>
</ul>
</p>
</body>
</html>

View File

@ -0,0 +1,35 @@
<!DOCTYPE html>
<html lang="en">
<!--
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.
-->
<head>
<meta charset="utf-8" />
<title>HashAttribute</title>
<link rel="stylesheet" href="../../css/component-usage.css" type="text/css" />
</head>
<body>
<!-- Processor Documentation ================================================== -->
<p>
<strong>Modifies Attributes:</strong>
</p>
<p>
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 &lt;Hash Value Attribute Key&gt; property.
</p>
</body>
</html>

View File

@ -1,88 +0,0 @@
<!DOCTYPE html>
<html lang="en">
<!--
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.
-->
<head>
<meta charset="utf-8" />
<title>HashAttribute</title>
<link rel="stylesheet" href="../../css/component-usage.css" type="text/css" />
</head>
<body>
<!-- Processor Documentation ================================================== -->
<h2> Description:</h2>
<p>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.
</p>
<p>
<strong>Modifies Attributes:</strong>
</p>
<p>
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 &lt;Hash Value Attribute Key&gt; property.
</p>
<p>
<strong>Properties:</strong>
</p>
<p>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.</p>
<ul>
<li><strong>Hash Value Attribute Key</strong>
<ul>
<li>The name of the FlowFile attribute whose hash value should
be stored.</li>
<li>Default value: no default</li>
<li>Supports expression language: false</li>
</ul></li>
<li>User-Added Properties
<ul>
<li>See description above.</li>
</ul>
</li>
</ul>
<p>
<strong>Relationships:</strong>
</p>
<ul>
<li>success
<ul>
<li>If the processor successfully hashes the key/value pairs of
the FlowFile, then the FlowFile follows this relationship.</li>
</ul>
</li>
<li>failure
<ul>
<li>If something prevents the processor from successfully
hashing the key/value pairs of the FlowFile, then the FlowFile
follows this relationship.</li>
</ul>
</li>
</ul>
</body>
</html>

View File

@ -0,0 +1,35 @@
<!DOCTYPE html>
<html lang="en">
<!--
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.
-->
<head>
<meta charset="utf-8" />
<title>HashContent</title>
<link rel="stylesheet" href="../../css/component-usage.css" type="text/css" />
</head>
<body>
<!-- Processor Documentation ================================================== -->
<p>
<strong>Modifies Attributes:</strong>
</p>
<p>
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 &lt;Hash Attribute Name&gt; property.
</p>
</body>
</html>

View File

@ -1,89 +0,0 @@
<!DOCTYPE html>
<html lang="en">
<!--
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.
-->
<head>
<meta charset="utf-8" />
<title>HashContent</title>
<link rel="stylesheet" href="../../css/component-usage.css" type="text/css" />
</head>
<body>
<!-- Processor Documentation ================================================== -->
<h2>Description:</h2>
<p>
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 <strong>Hash Attribute Name</strong>
property.
</p>
<p>
<strong>Modifies Attributes:</strong>
</p>
<p>
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 &lt;Hash Attribute Name&gt; property.
</p>
<p>
<strong>Properties:</strong>
</p>
<p>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.</p>
<ul>
<li><strong>Hash Attribute Name</strong>
<ul>
<li>The name of the FlowFile attribute to which the hash value
should be written. If the value already exists, it will be
overwritten.</li>
<li>Default value: hash.value</li>
<li>Supports expression language: false</li>
</ul></li>
<li><strong>Hash Algorithm</strong>
<ul>
<li>The hashing algorithm that should be used to perform the
hashing function.</li>
<li>Default value: MD5</li>
<li>Supports expression language: false</li>
</ul></li>
</ul>
</ul>
<p>
<strong>Relationships:</strong>
</p>
<ul>
<li>success
<ul>
<li>If the FlowFile is processed successfully, then the
FlowFile follows this relationship.</li>
</ul>
</li>
<li>failure
<ul>
<li>If the FlowFile is not processed successfully, then the
FlowFile follows this relationship.</li>
</ul>
</li>
</ul>
</body>
</html>

View File

@ -42,6 +42,7 @@
</p> </p>
<p>The following MIME Types are detected: <p>The following MIME Types are detected:
</p>
<ul> <ul>
<li>application/gzip</li> <li>application/gzip</li>
<li>application/bzip2</li> <li>application/bzip2</li>
@ -73,12 +74,12 @@
<li>application/zip</li> <li>application/zip</li>
<li>application/x-lzh</li> <li>application/x-lzh</li>
</ul> </ul>
</p>
<p> <p>
<strong>Modifies Attributes:</strong> <strong>Modifies Attributes:</strong>
</p> </p>
<table border="1"> <table>
<thead> <thead>
<tr> <tr>
<th>Attribute Name</th> <th>Attribute Name</th>
@ -93,44 +94,5 @@
</tr> </tr>
</tbody> </tbody>
</table> </table>
<p>
<strong>Properties:</strong>
</p>
<p>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.</p>
<ul>
<li><strong>Identify ZIP</strong>
<ul>
<li>A Boolean value (true/false), indicating whether to attempt in depth identification
of ZIP MIME types.</li>
<li>Default value: false</li>
<li>Supports expression language: false</li>
</ul></li>
<li><strong>Identify TAR</strong>
<ul>
<li>A Boolean value (true/false), indicating whether to attempt in depth identification
of TAR MIME types.</li>
<li>Default value: false</li>
<li>Supports expression language: false</li>
</ul></li>
</ul>
</ul>
<p>
<strong>Relationships:</strong>
</p>
<ul>
<li>success
<ul>
<li>All FlowFiles follow this relationship, regardless of
whether the MIME type was detected.</li>
</ul>
</li>
</ul>
</body> </body>
</html> </html>

View File

@ -0,0 +1,64 @@
<!DOCTYPE html>
<html lang="en">
<!--
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.
-->
<head>
<meta charset="utf-8" />
<title>InvokeHTTP - a swiss army http client processor</title>
<link rel="stylesheet" href="../../css/component-usage.css" type="text/css" />
</head>
<body>
<p>
<strong>Adds Attributes:</strong>
</p>
<table>
<thead>
<tr>
<th>Attribute Name</th>
<th>Description</th>
</tr>
</thead>
<tbody>
<tr>
<td>invokehttp.status.code</td>
<td>The status code that is returned.</td>
</tr>
<tr>
<td>invokehttp.status.message</td>
<td>The status message that is returned.</td>
</tr>
<tr>
<td>invokehttp.response.body</td>
<td>The response body.</td>
</tr>
<tr>
<td>invokehttp.request.url</td>
<td>The request URL.</td>
</tr>
<tr>
<td>invokehttp.tx.id</td>
<td>The transaction ID that is returned after reading the response.</td>
</tr>
<tr>
<td>invokehttp.remote.dn</td>
<td>The DN of the remote server.</td>
</tr>
</tbody>
</table>
</body>
</html>

View File

@ -1,181 +0,0 @@
<!DOCTYPE html>
<html lang="en">
<!--
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.
-->
<head>
<meta charset="utf-8" />
<title>InvokeHTTP - a swiss army http client processor</title>
<link rel="stylesheet" href="../../css/component-usage.css" type="text/css" />
</head>
<body>
<h2>Description:</h2>
<p>
Making requests to remote HTTP servers. Supporting common HTTP methods.
Storing results as new flowfiles upon success. Routing to failure on error.
</p>
<p>
An HTTP client processor that converts FlowFile attributes to HTTP headers with configurable HTTP method, URL, etc.
<p>
<p>
<strong>Adds Attributes:</strong>
</p>
<table border="1">
<thead>
<tr>
<th>Attribute Name</th>
<th>Description</th>
</tr>
</thead>
<tbody>
<tr>
<td>invokehttp.status.code</td>
<td>The status code that is returned.</td>
</tr>
<tr>
<td>invokehttp.status.message</td>
<td>The status message that is returned.</td>
</tr>
<tr>
<td>invokehttp.response.body</td>
<td>The response body.</td>
</tr>
<tr>
<td>invokehttp.request.url</td>
<td>The request URL.</td>
</tr>
<tr>
<td>invokehttp.tx.id</td>
<td>The transaction ID that is returned after reading the response.</td>
</tr>
<tr>
<td>invokehttp.remote.dn</td>
<td>The DN of the remote server.</td>
</tr>
</tbody>
</table>
<p>
<strong>Properties:</strong>
</p>
<ul>
<li>
<strong>HTTP Method</strong>
<ul>
<li>The HTTP request method (e.g., GET, POST, PUT, DELETE, HEAD, OPTIONS)</li>
<li>Default value: GET</li>
<li>Supports expression language: false</li>
</ul>
</li>
<li>
<strong>Remote URL</strong>
<ul>
<li>The remote URL that will be conneted to, including scheme, host, port, and path.</li>
<li>Default value: no default</li>
<li>Supports expression language: true</li>
</ul>
</li>
<li>
SSL Context Service
<ul>
<li>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.</li>
<li>Default value: no default</li>
<li>Supports expression language: false</li>
</ul>
</li>
<li>
<strong>Connection Timeout</strong>
<ul>
<li>The amount of time to wait before timing out while creating a connection.</li>
<li>Default value: 5 secs</li>
<li>Supports expression language: false</li>
</ul>
</li>
<li>
<strong>Read Timeout</strong>
<ul>
<li>The amount of time to wait before timing out while waiting for a response from the remote service.</li>
<li>Default value: 15 secs</li>
<li>Supports expression language: false</li>
</ul>
</li>
<li>
<strong>Include Data Header</strong>
<ul>
<li>A Boolean value (true/false), indicating whether to include an RFC-2616 date header in the request.</li>
<li>Default value: True</li>
<li>Supports expression language: false</li>
</ul>
</li>
<li>
<strong>Follow Redirects</strong>
<ul>
<li>A Boolean value (true/false), indicating whether to follow HTTP redirects issued by the remote server.</li>
<li>Default value: True</li>
<li>Supports expression language: false</li>
</ul>
</li>
<li>
Attributes to Send
<ul>
<li>A regular expression that defines which attributes to send as HTTP headers in the request. If not defined, no attributes are sent as headers.</li>
<li>Default value: no default</li>
<li>Supports expression language: false</li>
</ul>
</li>
</ul>
<p>
<strong>Relationships:</strong>
</p>
<ul>
<li>
<strong>Original</strong>
<ul>
<li>Original FlowFile will be routed upon success (2xx status codes).</li>
</ul>
</li>
<li>
<strong>Response</strong>
<ul>
<li>Response FlowFile will be routed upon success (2xx status codes).</li>
</ul>
</li>
<li>
<strong>Retry</strong>
<ul>
<li>FlowFile will be routed on any status code that can be retried (5xx status codes).</li>
</ul>
</li>
<li>
<strong>No Retry</strong>
<ul>
<li>FlowFile will be routed on any status code that should NOT be retried (1xx, 3xx, 4xx status codes).</li>
</ul>
</li>
<li>
<strong>Failure</strong>
<ul>
<li>FlowFile will be routed on any type of connection failure, timeout or general exception.</li>
</ul>
</li>
</ul>
</body>
</html>

View File

@ -1,86 +0,0 @@
<!DOCTYPE html>
<html lang="en">
<!--
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.
-->
<head>
<meta charset="utf-8" />
<title>ListenHTTP</title>
<link rel="stylesheet" href="../../css/component-usage.css" type="text/css" />
</head>
<body>
<!-- Processor Documentation ================================================== -->
<h2>Description:</h2>
<p>
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.
</p>
<p>
<strong>Properties:</strong>
</p>
<p>
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.
</p>
<ul>
<li><strong>Listening Port</strong>
<ul>
<li>The port to listen on for incoming connections.</li>
<li>Default value: no default</li>
<li>Supports expression language: false</li>
</ul></li>
<li><strong>Max Data to Receive per Second</strong>
<ul>
<li>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.</li>
<li>Default value: no default</li>
<li>Supports expression language: false</li>
</ul></li>
<li>SSL Context Service
<ul>
<li>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.</li>
<li>Default value: no default</li>
<li>Supports expression language: false</li>
</ul></li>
<li><strong>Authorized DN Pattern</strong>
<ul>
<li>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.</li>
<li>Default value: .*</li>
<li>Supports expression language: false</li>
</ul></li>
<li>HTTP Headers to receive as Attributes (Regex)
<ul>
<li>Specifies the Regular Expression that determines the names of HTTP Headers that should be passed along as FlowFile attributes</li>
<li>Default value: no default</li>
<li>Supports expression language: false</li>
</ul></li>
<li><strong>Max Unconfirmed FlowFile Time</strong>
<ul>
<li>The maximum amount of time to wait for a FlowFile to be confirmed before it is removed from the cache.</li>
<li>Default value: 60 secs</li>
<li>Supports expression language: false</li>
</ul></li>
</ul>
<p>
<strong>Relationships:</strong>
</p>
<ul>
<li>success
<ul>
<li>If the processor successfully receives files via an HTTP service, then the FlowFiles follow this relationship.</li>
</ul></li>
</ul>
</body>
</html>

View File

@ -1,144 +0,0 @@
<!DOCTYPE html>
<html lang="en">
<!--
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.
-->
<head>
<meta charset="utf-8" />
<title>ListenUDP</title>
<link rel="stylesheet" href="../../css/component-usage.css" type="text/css" />
</head>
<body>
<!-- Processor Documentation ================================================== -->
<h2>Description:</h2>
<p>
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.
</p>
<p>
<strong>Properties:</strong>
</p>
<p>
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.
</p>
<ul>
<li>
<strong>Port</strong>
<ul>
<li>The port to listen on for data packets. Must be known by senders of Datagrams. May be a system
property or environment variable.</li>
<li>Default value: no default</li>
<li>Supports expression language: false</li>
</ul>
</li>
<li>
<strong>Receive Timeout</strong>
<ul>
<li>The time out period when waiting to receive data from the socket. Specify units.</li>
<li>Default value: 5 secs</li>
<li>Supports expression language: false</li>
</ul>
</li>
<li>
<strong>Max Buffer Size</strong>
<ul>
<li>Determines the size each receive buffer may be. Specify units.</li>
<li>Default value: 1 MB</li>
<li>Supports expression language: false</li>
</ul>
</li>
<li>
<strong>FlowFile Size Trigger</strong>
<ul>
<li>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.</li>
<li>Default value: 1 MB </li>
<li>Supports expression language: false</li>
</ul>
</li>
<li>
<strong>Max size of UDP Buffer</strong>
<ul>
<li>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.</li>
<li>Default value: 1 MB</li>
<li>Supports expression language: false</li>
</ul>
</li>
<li>
<strong>Receive Buffer Count</strong>
<ul>
<li>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.</li>
<li>Default value: 4</li>
<li>Supports expression language: false</li>
</ul>
</li>
<li>
<strong>Channel Reader Interval</strong>
<ul>
<li>Scheduling interval for each read channel. Specify units.</li>
<li>Default value: 50 millisecs</li>
<li>Supports expression language: false</li>
</ul>
</li>
<li>
<strong>FlowFiles Per Session</strong>
<ul>
<li>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.</li>
<li>Default value: 10</li>
<li>Supports expression language: false</li>
</ul>
</li>
<li>
Sending Host
<ul>
<li>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.</li>
<li>Default value: none</li>
<li>Supports expression language: true</li>
</ul>
</li>
<li>
Sending Host Port
<ul>
<li>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.</li>
<li>Default value: none</li>
<li>Supports expression language: true</li>
</ul>
</li>
</ul>
<p>
<strong>Relationships:</strong>
</p>
<ul>
<li>
success
<ul>
<li>Used when file is successfully created and filled with UDP packets.</li>
</ul>
</li>
</ul>
</body>
</html>

View File

@ -1,80 +0,0 @@
<!DOCTYPE html>
<html lang="en">
<!--
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.
-->
<head>
<meta charset="utf-8" />
<title>LogAttribute</title>
<link rel="stylesheet" href="../../css/component-usage.css" type="text/css" />
</head>
<body>
<!-- Processor Documentation ================================================== -->
<h2>Description:</h2>
<p>
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.</p>
<p>
<strong>Properties:</strong>
</p>
<p>
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.
</p>
<ul>
<li>Attributes to Ignore
<ul>
<li>A comma-separated list of attributes to ignore. If not specified, no attributes will be ignored.</li>
<li>Default value: no default</li>
<li>Supports expression language: false</li>
</ul></li>
<li>Attributes to Log
<ul>
<li>A comma-separated list of attributes to log. If not specified, all attributes are logged.</li>
<li>Default value: no default</li>
<li>Supports expression language: false</li>
</ul></li>
<li><strong>Log Payload</strong>
<ul>
<li>A Boolean value (true/false), indicating whether to log the payload of incoming FlowFiles.</li>
<li>Default value: false</li>
<li>Supports expression language: false</li>
</ul></li>
<li><strong>Log Level</strong>
<ul>
<li>The log level at which to log the attributes. Acceptable values are:
<ul>
<li>trace</li>
<li>debug</li>
<li>info</li>
</ul></li>
<li>Default value: info</li>
<li>Supports expression language: false</li>
</ul></li>
</ul>
<p>
<strong>Relationships:</strong>
</p>
<ul>
<li>success
<ul>
<li>All outgoing FlowFiles follow this relationship.</li>
</ul></li>
</ul>
</body>
</html>

View File

@ -0,0 +1,115 @@
<!DOCTYPE html>
<html lang="en">
<!--
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.
-->
<head>
<meta charset="utf-8" />
<title>MergeContent</title>
<link rel="stylesheet" href="../../css/component-usage.css" type="text/css" />
</head>
<body>
<!-- Processor Documentation ================================================== -->
<p>
<strong>Uses Attributes:</strong>
</p>
<table>
<thead>
<tr>
<th>Attribute Name</th>
<th>Description</th>
</tr>
</thead>
<tbody>
<tr>
<td>fragment.identifier</td>
<td>Applicable only if the &lt;Merge Strategy&gt; property is set to <code>Defragment</code>.
All FlowFiles with the same value for this attribute will be bundled together.
</td>
</tr>
<tr>
<td>fragment.index</td>
<td>Applicable only if the &lt;Merge Strategy&gt; property is set to <code>Defragment</code>.
This attribute must be present on all FlowFiles with the same value for the <code>fragment.identifier</code>
attribute and must be a unique integer between 0 and the value of the <code>fragment.count</code> attribute.
This attribute indicates the order in which the fragments should be assembled.
</td>
</tr>
<tr>
<td>fragment.count</td>
<td>Applicable only if the &lt;Merge Strategy&gt; property is set to <code>Defragment</code>.
This attribute must be present on all FlowFiles with the same value for the <code>fragment.identifier</code>
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.
</td>
</tr>
<tr>
<td>segment.original.filename</td>
<td>Applicable only if the &lt;Merge Strategy&gt; property is set to <code>Defragment</code>.
This attribute must be present on all FlowFiles with the same value for the <code>fragment.identifier</code>
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.
</td>
</tr>
<tr>
<td>tar.permissions</td>
<td>Applicable only if the &lt;Merge Format&gt; property is set to <code>TAR</code>.
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.
</td>
</tr>
</tbody>
</table>
<p>
<strong>Modifies Attributes:</strong>
</p>
<table>
<thead>
<tr>
<th>Attribute Name</th>
<th>Description</th>
</tr>
</thead>
<tbody>
<tr>
<td>filename</td>
<td>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:
<ul>
<li>if Merge Format is TAR, then the filename will be appended with .tar</li>
<li>if Merge Format is ZIP, then the filename will be appended with .zip</li>
<li>if Merge Format is FlowFileStream, then the filename will be appended with .pkg</li>
</ul>
</td>
</tr>
<tr>
<td>merge.count</td>
<td>The number of FlowFiles that were merged into this bundle.</td>
</tr>
<tr>
<td>merge.bin.age</td>
<td>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.</td>
</tr>
</tbody>
</table>
</body>
</html>

View File

@ -1,347 +0,0 @@
<!DOCTYPE html>
<html lang="en">
<!--
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.
-->
<head>
<meta charset="utf-8" />
<title>MergeContent</title>
<link rel="stylesheet" href="../../css/component-usage.css" type="text/css" />
</head>
<body>
<!-- Processor Documentation ================================================== -->
<h2>Description:</h2>
<p>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.</p>
<p>
<strong>Uses Attributes:</strong>
</p>
<table border="1">
<thead>
<tr>
<th>Attribute Name</th>
<th>Description</th>
</tr>
</thead>
<tbody>
<tr>
<td>fragment.identifier</td>
<td>Applicable only if the &lt;Merge Strategy&gt; property is set to <code>Defragment</code>.
All FlowFiles with the same value for this attribute will be bundled together.
</td>
</tr>
<tr>
<td>fragment.index</td>
<td>Applicable only if the &lt;Merge Strategy&gt; property is set to <code>Defragment</code>.
This attribute must be present on all FlowFiles with the same value for the <code>fragment.identifier</code>
attribute and must be a unique integer between 0 and the value of the <code>fragment.count</code> attribute.
This attribute indicates the order in which the fragments should be assembled.
</td>
</tr>
<tr>
<td>fragment.count</td>
<td>Applicable only if the &lt;Merge Strategy&gt; property is set to <code>Defragment</code>.
This attribute must be present on all FlowFiles with the same value for the <code>fragment.identifier</code>
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.
</td>
</tr>
<tr>
<td>segment.original.filename</td>
<td>Applicable only if the &lt;Merge Strategy&gt; property is set to <code>Defragment</code>.
This attribute must be present on all FlowFiles with the same value for the <code>fragment.identifier</code>
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.
</td>
</tr>
<tr>
<td>tar.permissions</td>
<td>Applicable only if the &lt;Merge Format&gt; property is set to <code>TAR</code>.
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.
</td>
</tr>
</tbody>
</table>
<p>
<strong>Modifies Attributes:</strong>
</p>
<table border="1">
<thead>
<tr>
<th>Attribute Name</th>
<th>Description</th>
</tr>
</thead>
<tbody>
<tr>
<td>filename</td>
<td>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:
<ul>
<li>if Merge Format is TAR, then the filename will be appended with .tar</li>
<li>if Merge Format is ZIP, then the filename will be appended with .zip</li>
<li>if Merge Format is FlowFileStream, then the filename will be appended with .pkg</li>
</ul>
</td>
</tr>
<tr>
<td>merge.count</td>
<td>The number of FlowFiles that were merged into this bundle.</td>
</tr>
<tr>
<td>merge.bin.age</td>
<td>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.</td>
</tr>
</tbody>
</table>
<p>
<strong>Properties:</strong>
</p>
<p>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.</p>
<ul>
<li><strong>Merge Strategy</strong>
<ul>
<li>Specifies the algorithm used to merge content. The
following Merge Strategies are available:
<ul>
<li><b>Bin-Packing Algorithm</b> - This strategy uses the
&lt;Minimum Number of Entries&gt; and &lt;Maximum Number of
Entries&gt; properties to determine how many FlowFiles should be
used to create a Bundle.</li>
<li><b>Defragment</b> - This strategy is used when
recombining a set of FlowFiles that have been broken apart
previously. When using this value, the &lt;Minimum Number of
Entries&gt; and &lt;Maximum Number of Entries&gt; properties are
ignored. In lieu of using these properties, the pocessor
determines the number of FlowFiles to use for a bundle by
examining the <code>fragment.count</code> attribute of the
FlowFiles.</li>
</ul>
</li>
<li>Default value: Bin-Packing Algorithm</li>
<li>Supports expression language: false</li>
</ul></li>
<li><strong>Merge Format</strong>
<ul>
<li>Determines the format that will be used to merge the
content. Options include the following values:
<ul>
<li><b>Binary Concatenation</b> - FlowFiles will be combined
by concatenating their content. The ordering of the FlowFiles is
non-deterministic.</li>
<li><b>TAR</b> - FlowFiles will be combined by creating a TAR
file. If the &lt;Keep Path&gt; property is set to <code>true</code>,
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 <code>tar.permissions</code> 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.</li>
<li><b>ZIP</b> - FlowFiles will be combined by creating a ZIP
file. If the &lt;Keep Path&gt; property is set to <code>true</code>,
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 &lt;Compression Level&gt; property can be used to
determine the Compression Level to use, from 0 (no compression)
to 9 (highest compression).</li>
<li><b>FlowFileStream, v3</b> - 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.</li>
<li><b>FlowFileStream, v2</b> - 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.</li>
<li><b>FlowFile Tar, v1</b> - 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.</li>
</ul>
<li>Default value: Binary Concatenation</li>
<li>Supports expression language: false</li>
</ul>
</li>
<li><strong>Attribute Strategy</strong>
<ul>
<li>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.</li>
<li>Default value: Keep Only Common Attributes</li>
<li>Supports expression language: false</li>
</ul></li>
<li>Correlation Attribute Name
<ul>
<li>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.</li>
<li>Default value: no default</li>
<li>Supports expression language: false</li>
</ul>
</li>
<li><strong>Minimum Number of Entries</strong>
<ul>
<li>The minimum number of files to include in a bundle.</li>
<li>Default value: 1</li>
<li>Supports expression language: false</li>
</ul></li>
<li>Maximum Number of Entries
<ul>
<li>The maximum number of files to include in a bundle. If
not specified, there is no maximum.</li>
<li>Default value: no default</li>
<li>Supports expression language: false</li>
</ul>
</li>
<li><strong>Minimum Group Size</strong>
<ul>
<li>The minimum size for each bundle.</li>
<li>Default value: 0 B</li>
<li>Supports expression language: false</li>
</ul></li>
<li>Maximum Group Size
<ul>
<li>The maximum size for each bundle. If not specified, there
is no maximum.</li>
<li>Default value: no default</li>
<li>Supports expression language: false</li>
</ul>
</li>
<li>Max Bin Age
<ul>
<li>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.</li>
<li>Default value: no default</li>
<li>Supports expression language: false</li>
</ul>
</li>
<li><strong>Maximum Number of Bins</strong>
<ul>
<li>The maximum number of bins that can be held in memory at
any one time.</li>
<li>Default value: 100</li>
<li>Supports expression language: false</li>
</ul></li>
<li>Header File
<ul>
<li>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.</li>
<li>Default value: no default</li>
<li>Supports expression language: false</li>
</ul>
</li>
<li>Footer File
<ul>
<li>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.</li>
<li>Default value: no default</li>
<li>Supports expression language: false</li>
</ul>
</li>
<li>Demarcator File
<ul>
<li>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.</li>
<li>Default value: no default</li>
<li>Supports expression language: false</li>
</ul>
</li>
<li><strong>Compression Level</strong>
<ul>
<li>Specifies the compression level to use when using the ZIP
merge format. If not using the zip merge format, this value is
ignored.</li>
<li>Default value: 1</li>
<li>Supports expression language: false</li>
</ul></li>
<li><strong>Keep Path</strong>
<ul>
<li>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.</li>
<li>Default value: false</li>
<li>Supports expression language: false</li>
</ul></li>
</ul>
<p>
<strong>Relationships:</strong>
</p>
<ul>
<li>failure
<ul>
<li>If something prevents the processor from bundling the
incoming files, then they follow this relationship.</li>
</ul>
</li>
<li>merged
<ul>
<li>If the merging was successful, then the FlowFile
containing the merged content follows this relationship.</li>
</ul>
</li>
<li>original
<ul>
<li>If the merging was successful, then an original copy of
each file that was used to create a bundle follows this
relationship.</li>
</ul>
</li>
</ul>
</body>
</html>

View File

@ -1,64 +0,0 @@
<!DOCTYPE html>
<html lang="en">
<!--
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.
-->
<head>
<meta charset="utf-8" />
<title>ModifyBytes</title>
<link rel="stylesheet" href="../../css/component-usage.css" type="text/css" />
</head>
<body>
<!-- Processor Documentation ================================================== -->
<h2>Description:</h2>
<p>
This processor updates the content of a FlowFile by removing bytes from start or end of a file.
</p>
<p>
<strong>Properties:</strong>
</p>
<p>
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.
</p>
<ul>
<li><strong>Start Offset</strong>
<ul>
<li>Number of bytes removed at the beginning of the file.
</li>
<li>Default value: 0 B </li>
<li>Supports expression language: false</li>
</ul></li>
<li><strong>End Offset</strong>
<ul>
<li>Number of bytes removed at the end of the file.
</li>
<li>Default value: 0 B </li>
<li>Supports expression language: false</li>
</ul></li>
</ul>
<p>
<strong>Relationships:</strong>
</p>
<ul>
<li>success
<ul>
<li>If FlowFiles are successfully updated, then they follow this relationship.</li>
</ul>
</li>
</ul>
</body>
</html>

View File

@ -0,0 +1,49 @@
<!DOCTYPE html>
<html lang="en">
<!--
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.
-->
<head>
<meta charset="utf-8" />
<title>MonitorActivity</title>
<link rel="stylesheet" href="../../css/component-usage.css" type="text/css" />
</head>
<body>
<!-- Processor Documentation ================================================== -->
<p>
<strong>Modifies Attributes:</strong>
</p>
<table>
<thead>
<tr>
<th>Attribute Name</th>
<th>Description</th>
</tr>
</thead>
<tbody>
<tr>
<td>inactivityStartMillis</td>
<td>The time at which Inactivity began, in the form of milliseconds since Epoch.</td>
</tr>
<tr>
<td>inactivityDurationMillis</td>
<td>The number of milliseconds that the inactivity has spanned.</td>
</tr>
</tbody>
</table>
</body>
</html>

View File

@ -1,143 +0,0 @@
<!DOCTYPE html>
<html lang="en">
<!--
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.
-->
<head>
<meta charset="utf-8" />
<title>MonitorActivity</title>
<link rel="stylesheet" href="../../css/component-usage.css" type="text/css" />
</head>
<body>
<!-- Processor Documentation ================================================== -->
<h2>Description:</h2>
<p>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.</p>
<p>
<strong>Modifies Attributes:</strong>
</p>
<table border="1">
<thead>
<tr>
<th>Attribute Name</th>
<th>Description</th>
</tr>
</thead>
<tbody>
<tr>
<td>inactivityStartMillis</td>
<td>The time at which Inactivity began, in the form of milliseconds since Epoch.</td>
</tr>
<tr>
<td>inactivityDurationMillis</td>
<td>The number of milliseconds that the inactivity has spanned.</td>
</tr>
</tbody>
</table>
<p>
<strong>Properties:</strong>
</p>
<p>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.</p>
<ul>
<li><strong>Threshold Duration</strong>
<ul>
<li>The amount of time that must elapse before the flow is
considered inactive.</li>
<li>Default value: 5 min</li>
<li>Supports expression language: false</li>
</ul></li>
<li><strong>Continually Send Messages</strong>
<ul>
<li>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.</li>
<li>Default value: false</li>
<li>Supports expression language: false</li>
</ul></li>
<li><strong>Inactivity Message</strong>
<ul>
<li>The message that will appear as the content of outgoing
FlowFiles that are sent to the inactive relationship.</li>
<li>Default value:
<ul>
<li>Lacking activity as of time: ${now():format('yyyy/MM/dd
HH:mm:ss')}; flow has been inactive for
${inactivityDurationMillis:toNumber():divide(60000)} minutes.</li>
</ul>
</li>
<li>Supports expression language: true</li>
</ul></li>
<li><strong>Activity Restored Message</strong>
<ul>
<li>The message that will appear as the content of outgoing
FlowFiles that are sent to the activity.restored relationship.</li>
<li>Default value:
<ul>
<li>Activity restored at time: ${now():format('yyyy/MM/dd
HH:mm:ss')} after being inactive for
${inactivityDurationMillis:toNumber():divide(60000)} minutes.</li>
</ul>
</li>
<li>Supports expression language: true</li>
</ul></li>
<li>Copy Attributes
<ul>
<li>If true, will copy all flow file attributes from the flow file that resumed activity to the newly created indicator flow file.</li>
<li>Default value: false
</li>
<li>Supports expression language: false</li>
</ul></li>
</ul>
<p>
<strong>Relationships:</strong>
</p>
<ul>
<li>activity.restored
<ul>
<li>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.</li>
</ul>
</li>
<li>inactive
<ul>
<li>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.</li>
</ul>
</li>
<li>success
<ul>
<li>All incoming FlowFiles follow this relationship.</li>
</ul>
</li>
</ul>
</body>
</html>

View File

@ -0,0 +1,48 @@
<!DOCTYPE html>
<html lang="en">
<!--
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.
-->
<head>
<meta charset="utf-8" />
<title>PostHTTP</title>
<link rel="stylesheet" href="../../css/component-usage.css" type="text/css" />
</head>
<body>
<!-- Processor Documentation ================================================== -->
<p>
<strong>Uses Attributes:</strong>
</p>
<table>
<thead>
<tr>
<th>Attribute Name</th>
<th>Description</th>
</tr>
</thead>
<tbody>
<tr>
<td>mime.type</td>
<td>If not sending data as a FlowFile, the mime.type attribute will be used to set the HTTP Header for <code>Content-Type</code>.</td>
</tr>
</tbody>
</table>
</body>
</html>

View File

@ -1,187 +0,0 @@
<!DOCTYPE html>
<html lang="en">
<!--
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.
-->
<head>
<meta charset="utf-8" />
<title>PostHTTP</title>
<link rel="stylesheet" href="../../css/component-usage.css" type="text/css" />
</head>
<body>
<!-- Processor Documentation ================================================== -->
<h2>Description:</h2>
<p>This processor performs an HTTP post with the content of
each incoming FlowFile.
</p>
<p>
<strong>Uses Attributes:</strong>
</p>
<table border="1">
<thead>
<tr>
<th>Attribute Name</th>
<th>Description</th>
</tr>
</thead>
<tbody>
<tr>
<td>mime.type</td>
<td>If not sending data as a FlowFile, the mime.type attribute will be used to set the HTTP Header for <code>Content-Type</code>.</td>
</tr>
</tbody>
</table>
<p>
<strong>Properties:</strong>
</p>
<p>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.</p>
<ul>
<li><strong>URL</strong>
<ul>
<li>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.</li>
<li>Default value: no default</li>
<li>Supports expression language: true</li>
</ul></li>
<li>Max Batch Size
<ul>
<li>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.</li>
<li>Default value: 100 MB</li>
<li>Supports expression language: false</li>
</ul>
</li>
<li>Max Data To Post per Second
<ul>
<li>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.</li>
<li>Default value: no default</li>
<li>Supports expression language: false</li>
</ul>
</li>
<li>SSL Context Service
<ul>
<li>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.</li>
<li>Default value: no default</li>
<li>Supports expression language: false</li>
</ul>
</li>
<li>Username
<ul>
<li>The username required to access the URL.</li>
<li>Default value: no default</li>
<li>Supports expression language: false</li>
</ul>
</li>
<li>Password
<ul>
<li>The password required to access the URL.</li>
<li>Default value: no default</li>
<li>Supports expression language: false</li>
</ul>
</li>
<li><strong>Send as FlowFile</strong>
<ul>
<li>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.</li>
<li>Default value: false</li>
<li>Supports expression language: false</li>
</ul></li>
<li><strong>Use Chunked Encoding</strong>
<ul>
<li>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.</li>
<li>Default value: true</li>
<li>Supports expression language: false</li>
</ul></li>
<li><strong>Compression Level</strong>
<ul>
<li>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.</li>
<li>Default value: 0</li>
<li>Supports expression language: false</li>
</ul></li>
<li><strong>Connection Timeout</strong>
<ul>
<li>The amount of time to wait before timing out while creating
a connection.</li>
<li>Default value: 30 sec</li>
<li>Supports expression language: false</li>
</ul></li>
<li><strong>Data Timeout</strong>
<ul>
<li>The amount of time to wait before timing out while
transferring data.</li>
<li>Default value: 30 sec</li>
<li>Supports expression language: false</li>
</ul></li>
<li>Attributes to Send as HTTP Headers (Regex)
<ul>
<li>Specifies the regular expression that determines the names
of FlowFile attributes that should be sent as HTTP headers.</li>
<li>Default value: no default</li>
<li>Supports expression language: false</li>
</ul>
</li>
<li>User Agent
<ul>
<li>What to report as the user agent when a connection is made
to the remote server.</li>
<li>Default value: no default</li>
<li>Supports expression language: false</li>
</ul>
</li>
</ul>
<p>
<strong>Relationships:</strong>
</p>
<ul>
<li>failure
<ul>
<li>If something prevents the processor from successfully
posting the FlowFile, then the FlowFile follows this relationship.</li>
</ul>
</li>
<li>success
<ul>
<li>If the processor successfully posts the FlowFile, then it
follows this relationship.</li>
</ul>
</li>
</ul>
</body>
</html>

View File

@ -1,114 +0,0 @@
<!DOCTYPE html>
<html lang="en">
<!--
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.
-->
<head>
<meta charset="utf-8" />
<title>PutEmail</title>
<link rel="stylesheet" href="../../css/component-usage.css" type="text/css" />
</head>
<body>
<!-- Processor Documentation ================================================== -->
<h2>Description:</h2>
<p>
This processor sends an e-mail to configured recipients for each incoming FlowFile.</p>
<p>
<strong>Properties:</strong>
</p>
<p>
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.
</p>
<ul>
<li><strong>SMTP Hostname</strong>
<ul>
<li>The hostname of the SMTP host.</li>
<li>Default value: no default</li>
<li>Supports expression language: false</li>
</ul></li>
<li><strong>SMTP Port</strong>
<ul>
<li>The port used for SMTP communication.</li>
<li>Default value: 25</li>
<li>Supports expression language: false</li>
</ul></li>
<li><strong>From</strong>
<ul>
<li>Specifies the email address to use as the sender.</li>
<li>Default value: no default</li>
<li>Supports expression language: true</li>
</ul></li>
<li>To
<ul>
<li>The recipients to include in the To line of the email.</li>
<li>Default value: no default</li>
<li>Supports expression language: true</li>
</ul></li>
<li>CC
<ul>
<li>The recipients to include in the CC line of the email.</li>
<li>Default value: no default</li>
<li>Supports expression language: true</li>
</ul></li>
<li>BCC
<ul>
<li>The recipients to include in the BCC line of the email.</li>
<li>Default value: no default</li>
<li>Supports expression language: true</li>
</ul></li>
<li><strong>Subject</strong>
<ul>
<li>This content will appear in the email Subject line.</li>
<li>Default value: Message from NiFi</li>
<li>Supports expression language: true</li>
</ul></li>
<li><strong>Message</strong>
<ul>
<li>The body of the email message.</li>
<li>Default value: no default</li>
<li>Supports expression language: true</li>
</ul></li>
<li><strong>Attach File</strong>
<ul>
<li>A Boolean value (true/false), indicating whether to attach the FlowFile content to the email.</li>
<li>Default value: false</li>
<li>Supports expression language: false</li>
</ul></li>
<li><strong>Include All Attributes In Message</strong>
<ul>
<li>A Boolean value (true/false), indicating whether or not all FlowFile attributes should be recorded
in the body of the email message.</li>
<li>Default value: false</li>
<li>Supports expression language: false</li>
</ul></li>
</ul>
<p>
<strong>Relationships:</strong>
</p>
<ul>
<li>failure
<ul>
<li>If FlowFiles fail to send, then they follow this relationship.</li>
</ul></li>
<li>success
<ul>
<li>If an email is successfully sent for a given FlowFile, then the FlowFile follows this relationship.</li>
</ul></li>
</ul>
</body>
</html>

View File

@ -0,0 +1,66 @@
<!DOCTYPE html>
<html lang="en">
<!--
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.
-->
<head>
<meta charset="utf-8" />
<title>PutFTP</title>
<link rel="stylesheet" href="../../css/component-usage.css" type="text/css" />
</head>
<body>
<!-- Processor Documentation ================================================== -->
<p>
<strong>Optional User-Defined Properties:</strong>
</p>
<ul>
<li>pre.cmd.#
<ul>
<li>Optional properties of this type can be added by the user and should be used for <strong>VERY
RARE</strong> 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.
</li>
<li><strong>NOTE</strong>: 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.</li>
<li><strong>NOTE</strong>: If the optional property name does not follow the pattern pre.cmd.<em>integer</em>
- then the command will not be sent. If a command is given which is not recognized by the server, then
that will be logged.</li>
<li>Supports expression language: true. FlowFile attributes can be used in commands using the expression language</li>
</ul>
</li>
<li>post.cmd.#
<ul>
<li>Optional properties of this type can be added by the user and should be used for <strong>VERY
RARE</strong> 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.
</li>
<li>Supports expression language: true. FlowFile attributes can be used in commands using the expression language</li>
</ul>
</li>
</ul>
</body>
</html>

View File

@ -1,283 +0,0 @@
<!DOCTYPE html>
<html lang="en">
<!--
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.
-->
<head>
<meta charset="utf-8" />
<title>PutFTP</title>
<link rel="stylesheet" href="../../css/component-usage.css" type="text/css" />
</head>
<body>
<!-- Processor Documentation ================================================== -->
<h2>Description:</h2>
<p>
This processor sends FlowFiles via FTP to an FTP server.
</p>
<p>
<strong>Properties:</strong>
</p>
<p>
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.
</p>
<ul>
<li>
<strong>Hostname</strong>
<ul>
<li>The fully qualified hostname or IP address of the remote
system.</li>
<li>Default value: no default</li>
<li>Supports expression language: true</li>
</ul>
</li>
<li>
<strong>Port</strong>
<ul>
<li>The port that the remote system is listening on for file transfers.</li>
<li>Default value: 21</li>
<li>Supports expression language: false</li>
</ul>
</li>
<li>
<strong>Username</strong>
<ul>
<li>The username for the user account.</li>
<li>Default value: no default</li>
<li>Supports expression language: false</li>
</ul>
</li>
<li>
Password
<ul>
<li>The password for the user account.</li>
<li>Default value: no default</li>
<li>Supports expression language: false</li>
</ul>
</li>
<li>
Remote Path
<ul>
<li>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}.</li>
<li>Default value: no default</li>
<li>Supports expression language: true</li>
</ul>
</li>
<li><strong>Create Directory</strong>
<ul>
<li>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.</li>
<li>Default value: false</li>
<li>Supports expression language: false</li>
</ul>
</li>
<li>
<strong>Batch Size</strong>
<ul>
<li>The maximum number of FlowFiles to send in a single connection.</li>
<li>Default value: 500</li>
<li>Supports expression language: false</li>
</ul>
</li>
<li>
<strong>Connection Timeout</strong>
<ul>
<li>The amount of time to wait before timing out while creating a
connection.</li>
<li>Default value: 30 sec</li>
<li>Supports expression language: false</li>
</ul>
</li>
<li>
<strong>Data Timeout</strong>
<ul>
<li>The amount of time to wait before timing out while transferring
data.</li>
<li>Default value: 30 sec</li>
<li>Supports expression language: false</li>
</ul>
</li>
<li>
<strong>Conflict Resolution</strong>
<ul>
<li>
Specifies what action the processor should take if a conflict
prevents it from delivering the files. Valid options are:
<ul>
<li>replace</li>
<li>ignore</li>
<li>rename</li>
<li>reject</li>
<li>fail</li>
<li>NONE</li>
</ul>
</li>
<li>Default value: NONE</li>
<li>Supports expression language: false</li>
</ul>
</li>
<li>
Dot Rename
<ul>
<li>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.</li>
<li>Default value: true</li>
<li>Supports expression language: false</li>
</ul>
</li>
<li>
Temporary Filename
<ul>
<li>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.
</li>
<li>Default value: no default</li>
<li>Supports expression language: true</li>
</ul>
</li>
<li>
Transfer Mode
<ul>
<li>
The FTP transfer mode. Valid options are: Binary or ASCII.
</li>
<li>Default value: Binary</li>
<li>Supports expression language: false</li>
</ul>
</li>
<li>
Connection Mode
<ul>
<li>
The FTP connection mode. Valid options are: Active or Passive.
For most clients, this should be set to Passive.</li>
<li>Default value: Passive</li>
<li>Supports expression language: false</li>
</ul>
</li>
<li>
Reject Zero-Byte Files
<ul>
<li>A Boolean value (true/false), indicating whether to reject
files that have zero bytes of content rather than transferring
them. </li>
<li>Default value: true</li>
<li>Supports expression language: false</li>
</ul>
</li>
<li>Last Modified Time
<ul>
<li>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.</li>
<li>Default value: no default</li>
<li>Supports expression language: true</li>
</ul>
</li>
<li>Permissions
<ul>
<li>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.
</li>
<li>Default value: no default</li>
<li>Supports expression language: true</li>
</ul>
</li>
<li>
<strong>Use Compression</strong>
<ul>
<li>A Boolean value (true/false), indicating whether to use ZLIB
compression when transferring files.</li>
<li>Default value: false</li>
<li>Supports expression language: false</li>
</ul>
</li>
</ul>
<p>
<strong>Optional User-Defined Properties:</strong>
</p>
<p>
<ul>
<li>pre.cmd.#
<ul>
<li>Optional properties of this type can be added by the user and should be used for <strong>VERY
RARE</strong> 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.
</li>
<li><strong>NOTE</strong>: 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.</li>
<li><strong>NOTE</strong>: If the optional property name does not follow the pattern pre.cmd.<em>integer</em>
- then the command will not be sent. If a command is given which is not recognized by the server, then
that will be logged.</li>
<li>Supports expression language: true. FlowFile attributes can be used in commands using the expression language</li>
</ul>
</li>
<li>post.cmd.#
<ul>
<li>Optional properties of this type can be added by the user and should be used for <strong>VERY
RARE</strong> 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.
</li>
<li>Supports expression language: true. FlowFile attributes can be used in commands using the expression language</li>
</ul>
</li>
</ul>
</p>
<p>
<strong>Relationships:</strong>
</p>
<ul>
<li>
failure
<ul>
<li>If something prevents a FlowFile from being transferred, then
it follows this relationship.</li>
</ul>
</li>
<li>
reject
<ul>
<li>FlowFiles that were rejected by the destination system follow this relationship.</li>
</ul>
</li>
<li>
success
<ul>
<li>If a FlowFile is successfully transferred, then it follows this
relationship.</li>
</ul>
</li>
</ul>
</body>
</html>

View File

@ -1,109 +0,0 @@
<!DOCTYPE html>
<html lang="en">
<!--
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.
-->
<head>
<meta charset="utf-8" />
<title>PutFile</title>
<link rel="stylesheet" href="../../css/component-usage.css" type="text/css" />
</head>
<body>
<!-- Processor Documentation ================================================== -->
<h2>Description:</h2>
<p>
This processor writes FlowFiles to the local file system.</p>
<p>
<strong>Properties:</strong>
</p>
<p>
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.
</p>
<ul>
<li><strong>Directory</strong>
<ul>
<li>The local file path to the location where files should be written. You may use expression language such as /aa/bb/${path}.</li>
<li>Default value: no default</li>
<li>Supports expression language: true</li>
</ul></li>
<li><strong>Keep Directory Structure</strong>
<ul>
<li>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.
<b>Note:</b> 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 <code>/aa/bb directory</code>, for example, the <code>Directory</code> property should
instead be set to <code>/aa/bb/${path}</code>.
</li>
<li>Default value: false</li>
<li>Supports expression language: false</li>
</ul></li>
<li><strong>Conflict Resolution Strategy</strong>
<ul>
<li>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:
<ul>
<li>replace</li>
<li>ignore</li>
<li>fail</li>
</ul></li>
<li>Default value: fail</li>
<li>Supports expression language: false</li>
</ul></li>
<li>Maximum File Count
<ul>
<li>The maximum number of files that can exist in the output directory.</li>
<li>Default value: no default</li>
<li>Supports expression language: false</li>
</ul></li>
<li>Last Modified Time
<ul>
<li>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}.</li>
<li>Default value: no default</li>
<li>Supports expression language: true</li>
</ul></li>
<li>Permissions
<ul>
<li>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}.</li>
<li>Default value: no default</li>
<li>Supports expression language: true</li>
</ul></li>
<li>Owner
<ul>
<li>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.</li>
<li>Default value: no default</li>
<li>Supports expression language: true</li>
</ul></li>
<li>Group
<ul>
<li>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.</li>
<li>Default value: no default</li>
<li>Supports expression language: true</li>
</ul></li>
</ul>
<p>
<strong>Relationships:</strong>
</p>
<ul>
<li>failure
<ul>
<li>If something prevents a FlowFile from being written to its destination directory, then it follows this relationship.</li>
</ul></li>
<li>success
<ul>
<li>If a FlowFile is successfully written to its destination directory, then it follows this relationship.</li>
</ul></li>
</ul>
</body>
</html>

View File

@ -1,152 +0,0 @@
<!DOCTYPE html>
<html lang="en">
<!--
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.
-->
<head>
<meta charset="utf-8" />
<title>PutJMS</title>
<link rel="stylesheet" href="../../css/component-usage.css" type="text/css" />
</head>
<body>
<!-- Processor Documentation ================================================== -->
<h2>Description:</h2>
<p>
This processor creates a JMS message from the contents of a FlowFile and sends the message to a JMS server.</p>
<p>
<strong>Properties:</strong>
</p>
<p>
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.
</p>
<ul>
<li><strong>JMS Provider</strong>
<ul>
<li>This property specifies the provider used for the JMS server. Available options include:
<ul>
<li>ActiveMQ</li>
</ul></li>
<li>Default value: ActiveMQ</li>
<li>Supports expression language: false</li>
</ul></li>
<li><strong>URL</strong>
<ul>
<li>The URL of the JMS server.</li>
<li>Default value: no default</li>
<li>Supports expression language: false</li>
</ul></li>
<li><strong>Destination Name</strong>
<ul>
<li>The name of the JMS topic of queue to use.</li>
<li>Default value: no default</li>
<li>Supports expression language: false</li>
</ul></li>
<li><strong>Destination Type</strong>
<ul>
<li>The type of JMS Destination to use. Options are Queue and Topic.</li>
<li>Default value: Queue</li>
<li>Supports expression language: false</li>
</ul></li>
<li><strong>Communications Timeout</strong>
<ul>
<li>The amount of time to wait when attempting to receive a message before giving up and assuming failure.</li>
<li>Default value: 30 sec</li>
<li>Supports expression language: false</li>
</ul></li>
<li><strong>Batch Size</strong>
<ul>
<li>The number of messages to Put in a single iteration of the processor</li>
<li>Default value: 10</li>
<li>Supports expression language: false</li>
</ul></li>
<li>Username
<ul>
<li>The username used for authentication and authorization.</li>
<li>Default value: no default</li>
<li>Supports expression language: false</li>
</ul></li>
<li>Password
<ul>
<li>The password used for authentication and authorization.</li>
<li>Default value: no default</li>
<li>Supports expression language: false</li>
</ul></li>
<li><strong>Message Type</strong>
<ul>
<li>The type of JMS message to construct. Available options include:
<ul>
<li>byte</li>
<li>stream</li>
<li>text</li>
<li>empty</li>
</ul></li>
<li>Default value: byte</li>
<li>Supports expression language: false</li>
</ul></li>
<li>Message Priority
<ul>
<li>The priority of the message.</li>
<li>Default value: no default</li>
<li>Supports expression language: true</li>
</ul></li>
<li>Reply-to Queue
<ul>
<li>The name of the queue to which a reply should be added.</li>
<li>Default value: no default</li>
<li>Supports expression language: true</li>
</ul></li>
<li><strong>Max Buffer Size</strong>
<ul>
<li>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. </li>
<li>Default value: 1 MB</li>
<li>Supports expression language: false</li>
</ul></li>
<li>Message Time to Live
<ul>
<li>The amount of time that the message should live on the destination before being removed; if not specified, the message will never expire.</li>
<li>Default value: no default</li>
<li>Supports expression language: false</li>
</ul></li>
<li>Copy Attributes to JMS Properties
<ul>
<li>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. </li>
<li>Default value: true</li>
<li>Supports expression language: false</li>
</ul></li>
<li>Client ID Prefix
<ul>
<li>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.</li>
<li>Default value: no default</li>
<li>Supports expression language: false</li>
</ul></li>
</ul>
<p>
<strong>Relationships:</strong>
</p>
<ul>
<li>failure
<ul>
<li>If something prevents a FlowFile from being routed to the JMS destination, then it follows this relationship.</li>
</ul></li>
<li>success
<ul>
<li>If a FlowFile is successfully routed to the JMS destination, then it follows this relationship.</li>
</ul></li>
</ul>
</body>
</html>

View File

@ -0,0 +1,34 @@
<!DOCTYPE html>
<html lang="en">
<!--
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.
-->
<head>
<meta charset="utf-8" />
<title>PutSFTP</title>
<link rel="stylesheet" href="../../css/component-usage.css" type="text/css" />
</head>
<body>
<!-- Processor Documentation ================================================== -->
<p>
<strong>See Also:</strong>
</p>
<ul>
<li>
<a href="../org.apache.nifi.processors.standard.GetSFTP/index.html">GetSFTP</a>
</li>
</ul>
</body>
</html>

View File

@ -1,281 +0,0 @@
<!DOCTYPE html>
<html lang="en">
<!--
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.
-->
<head>
<meta charset="utf-8" />
<title>PutSFTP</title>
<link rel="stylesheet" href="../../css/component-usage.css" type="text/css" />
</head>
<body>
<!-- Processor Documentation ================================================== -->
<h2>Description:</h2>
<p>
This processor sends FlowFiles via SFTP to an SFTP server.
</p>
<p>
<strong>Properties:</strong>
</p>
<p>
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.
</p>
<ul>
<li>
<strong>Hostname</strong>
<ul>
<li>The fully qualified hostname or IP address of the remote
system.</li>
<li>Default value: no default</li>
<li>Supports expression language: true</li>
</ul>
</li>
<li>
<strong>Port</strong>
<ul>
<li>The port that the remote system is listening on for file transfers.</li>
<li>Default value: 22</li>
<li>Supports expression language: false</li>
</ul>
</li>
<li>
<strong>Username</strong>
<ul>
<li>The username for the user account.</li>
<li>Default value: no default</li>
<li>Supports expression language: false</li>
</ul>
</li>
<li>
Password
<ul>
<li>The password for the user account.</li>
<li>Default value: no default</li>
<li>Supports expression language: false</li>
</ul>
</li>
<li>
Private Key Path
<ul>
<li>The fully qualified path for the private key file.</li>
<li>Default value: no default</li>
<li>Supports expression language: false</li>
</ul>
</li>
<li>
Private Key Passphrase
<ul>
<li>The password for the private key.</li>
<li>Default value: no default</li>
<li>Supports expression language: false</li>
</ul>
</li>
<li>
Remote Path
<ul>
<li>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}.</li>
<li>Default value: no default</li>
<li>Supports expression language: true</li>
</ul>
</li>
<li><strong>Create Directory</strong>
<ul>
<li>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.</li>
<li>Default value: false</li>
<li>Supports expression language: false</li>
</ul>
</li>
<li><strong>Batch Size</strong>
<ul>
<li>The maximum number of FlowFiles to send in a single connection.</li>
<li>Default value: 500</li>
<li>Supports expression language: false</li>
</ul>
</li>
<li>
<strong>Connection Timeout</strong>
<ul>
<li>The amount of time to wait before timing out while creating a
connection. </li>
<li>Default value: 30 sec</li>
<li>Supports expression language: false</li>
</ul>
</li>
<li>
<strong>Data Timeout</strong>
<ul>
<li>The amount of time to wait before timing out while transferring
data. </li>
<li>Default value: 30 sec</li>
<li>Supports expression language: false</li>
</ul>
</li>
<li>
<strong>Conflict Resolution</strong>
<ul>
<li>Specifies what action the processor should take if a conflict
prevents it from delivering the files. Valid options are:</li>
<ul>
<li>replace</li>
<li>ignore</li>
<li>rename</li>
<li>reject</li>
<li>fail</li>
<li>NONE</li>
</ul>
<li>Default value: NONE</li>
<li>Supports expression language: false</li>
</ul>
</li>
<li>
Reject Zero-Byte Files
<ul>
<li>A Boolean value (true/false), indicating whether to reject
files that have zero bytes of content rather than transferring
them.</li>
<li>Default value: true</li>
<li>Supports expression language: false</li>
</ul>
</li>
<li>
Dot Rename
<ul>
<li>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.</li>
<li>Default value: true</li>
<li>Supports expression language: false</li>
</ul>
</li>
<li>Temporary Filename
<ul>
<li>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.
</li>
<li>Default value: no default</li>
<li>Supports expression language: true</li>
</ul>
</li>
<li>
Host Key File
<ul>
<li>The local file path to the host key file; if not supplied, no
host key file will be used. </li>
<li>Default value: no default</li>
<li>Supports expression language: false</li>
</ul>
</li>
<li>Last Modified Time
<ul>
<li>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.</li>
<li>Default value: no default</li>
<li>Supports expression language: true</li>
</ul>
</li>
<li>Permissions
<ul>
<li>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.
</li>
<li>Default value: no default</li>
<li>Supports expression language: true</li>
</ul>
</li>
<li>Remote Owner
<ul>
<li>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.</li>
<li>Default value: no default</li>
<li>Supports expression language: true</li>
</ul>
</li>
<li>Remote Group
<ul>
<li>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.</li>
<li>Default value: no default</li>
<li>Supports expression language: true</li>
</ul>
</li>
<li>
<strong>Strict Host Key Checking</strong>
<ul>
<li>A Boolean value (true/false), indicating whether to apply
strict enforcement of host keys. </li>
<li>Default value: false</li>
<li>Supports expression language: false</li>
</ul>
</li>
<li>
<strong>Use Compression</strong>
<ul>
<li>A Boolean value (true/false), indicating whether to use ZLIB
compression when transferring files.</li>
<li>Default value: false</li>
<li>Supports expression language: false</li>
</ul>
</li>
</ul>
<p>
<strong>Relationships:</strong>
</p>
<ul>
<li>
failure
<ul>
<li>If FlowFiles fail to be transferred, then they follow this
relationship.</li>
</ul>
</li>
<li>
reject
<ul>
<li>If the property Reject Zero-Byte Files is set to true and
incoming FlowFiles meet this criteria, then they follow this
relationship.</li>
</ul>
</li>
<li>
success
<ul>
<li>If FlowFiles are successfully transferred, then they follow
this relationship.</li>
</ul>
</li>
</ul>
<p>
<strong>See Also:</strong>
<ul>
<li>
<a href="../org.apache.nifi.processors.standard.GetSFTP/index.html">GetSFTP</a>
</li>
</ul>
</p>
</body>
</html>

View File

@ -1,91 +0,0 @@
<!DOCTYPE html>
<html lang="en">
<!--
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.
-->
<head>
<meta charset="utf-8" />
<title>ReplaceText</title>
<link rel="stylesheet" href="../../css/component-usage.css" type="text/css" />
</head>
<body>
<!-- Processor Documentation ================================================== -->
<h2>Description:</h2>
<p>
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.
</p>
<p>
<strong>Properties:</strong>
</p>
<p>
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.
</p>
<ul>
<li><strong>Regular Expression</strong>
<ul>
<li>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 <code>X</code> with the value <code>H.*o</code> and want it to match against "Hello", you cannot do this by referencing <code>${X}</code>. Rather, a reference to <code>${X}</code>
will reference the exact literal <code>H.*o</code>
</li>
<li>Default value: (.*) </li>
<li>Supports expression language: true</li>
</ul></li>
<li><strong>Replacement Value</strong>
<ul>
<li>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.</li>
<li>Default value: $1</li>
<li>Supports expression language: true</li>
</ul></li>
<li><strong>Character Set</strong>
<ul>
<li>The character set in which the file is encoded.</li>
<li>Default value: UTF-8</li>
<li>Supports expression language: false</li>
</ul></li>
<li><strong>Evaluation Mode</strong>
<ul>
<li>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'.</li>
<li>Default value: Entire text</li>
<li>Supports expression language: false</li>
</ul></li>
<li>Maximum Buffer Size
<ul>
<li>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 '.*'
</li>
<li>Default value: 1MB</li>
<li>Supports expression language: false</li>
</ul></li>
</ul>
<p>
<strong>Relationships:</strong>
</p>
<ul>
<li>failure
<ul>
<li>If FlowFiles cannot be updated, then they follow this relationship.</li>
</ul></li>
<li>success
<ul>
<li>If FlowFiles are successfully updated, then they follow this relationship.</li>
</ul></li>
</ul>
</body>
</html>

View File

@ -1,114 +0,0 @@
<!DOCTYPE html>
<html lang="en">
<!--
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.
-->
<head>
<meta charset="utf-8"><title>ReplaceText</title>
<link rel="stylesheet" href="../../css/component-usage.css" type="text/css">
</head>
<body>
<!-- Processor Documentation ================================================== -->
<h2>Description:</h2>
<p>
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&nbsp; group of the
regular expression with an alternate,
user-defined, value provided in a mapping file. &nbsp;The mapping
file is formatted as one key/value pair per line, seperated by tabs.
</p>
<p>
<strong>Properties:</strong>
</p>
<p>
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.
</p>
<ul>
<li><strong>Regular Expression</strong>
<ul>
<li>The regular expression to use to evaluate the FlowFile
content.</li>
<li>Default value: (\\S+) </li>
<li>Supports expression language: true</li>
</ul>
</li>
<li><strong>Matching Group</strong>
<ul>
<li>The number of the matching group of the provided regex
to replace
with the corresponding value from the mapping file (if it exists).</li>
<li>Default value: 0</li>
<li>Supports expression language: true</li>
</ul>
</li>
<li><strong>Mapping File</strong>
<ul>
<li>The name of the file (including the full path)
containing the Mappings.</li>
<li>Default value: no default</li>
<li>Supports expression language: false</li>
</ul>
</li>
<li><strong>Mapping File Refresh Interval</strong>
<ul>
<li>The polling interval in seconds to check for updates to
the mapping file.</li>
<li>Default value: 60s</li>
<li>Supports expression language: false</li>
</ul>
</li>
<li><strong>Character Set</strong>
<ul>
<li>The character set in which the file is encoded.</li>
<li>Default value: UTF-8</li>
<li>Supports expression language: false</li>
</ul>
</li>
<li><strong>Maximum Buffer Size</strong>
<ul>
<li>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.</li>
<li>Default value: 1MB</li>
<li>Supports expression language: false</li>
</ul>
</li>
</ul>
<p>
<strong>Relationships:</strong>
</p>
<ul>
<li>failure
<ul>
<li>If FlowFiles cannot be updated, then they follow this
relationship.</li>
</ul>
</li>
<li>success
<ul>
<li>If FlowFiles are successfully updated, then they follow
this relationship.</li>
</ul>
</li>
</ul>
</body></html>

View File

@ -0,0 +1,46 @@
<!DOCTYPE html>
<html lang="en">
<!--
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.
-->
<head>
<meta charset="utf-8" />
<title>RouteOnAttribute</title>
<link rel="stylesheet" href="../../css/component-usage.css" type="text/css" />
</head>
<body>
<h2>Description:</h2>
<p>
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).
</p>
<p>
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:
</p>
<ul>
<li><strong>property name</strong>: ABC
</li>
<li><strong>property value</strong>: ${filename:startsWith('ABC')}
</li>
</ul>
<p>
In this example, all files with filenames that start with ABC will follow the ABC relationship.
</p>
</body>
</html>

View File

@ -1,110 +0,0 @@
<!DOCTYPE html>
<html lang="en">
<!--
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.
-->
<head>
<meta charset="utf-8" />
<title>RouteOnAttribute</title>
<link rel="stylesheet" href="../../css/component-usage.css" type="text/css" />
</head>
<body>
<h2>Description:</h2>
<p>
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).
</p>
<p>
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:
</p>
<ul>
<li><strong>property name</strong>: ABC
</li>
<li><strong>property value</strong>: ${filename:startsWith('ABC')}
</li>
</ul>
<p>
In this example, all files with filenames that start with ABC will follow the ABC relationship.
</p>
<p>
<strong>Properties:</strong>
</p>
<p>
In the list below, the names of required properties appear
in bold.
</p>
<ul>
<li><strong>Routing Strategy</strong>
<ul>
<li>Specifies how to determine which relationship to use when
evaluating the expression language. Options are:
<ul>
<li><b>Route to Property Name</b> - For each property that has been added, if its Expression evaluates to <code>true</code>
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 <code>true</code>, a clone of the FlowFile will be created
for each relationship.</li>
<li><b>Route to 'match' if all match</b> - If all configured Expressions evaluate to <code>true</code>
for a given FlowFile, that FlowFile will be routed to 'matched'; otherwise, the FlowFile will be
routed to 'unmatched'.</li>
<li><b>Route to 'match' if any matches</b> - If any configured Expression evaluates to <code>true</code>
for a given FlowFile, that FlowFile will be routed to 'matched'; otherwise, the FlowFile will be
routed to 'unmatched'.</li>
</ul>
</li>
<li>Default value: Route to Property Name</li>
<li>Supports expression language: false</li>
</ul></li>
</ul>
<p>
<strong>Relationships:</strong>
</p>
<ul>
<li>matched
<ul>
<li>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.</li>
</ul>
</li>
<li>unmatched
<ul>
<li>FlowFiles that do not match any user-defined expression
will be routed to this relationship.</li>
</ul>
</li>
<li>user-added relationships
<ul>
<li>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).</li>
</ul>
</li>
</ul>
</body>
</html>

View File

@ -1,82 +0,0 @@
<!DOCTYPE html>
<html lang="en">
<!--
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.
-->
<head>
<meta charset="utf-8" />
<title>RouteOnContent</title>
<link rel="stylesheet" href="../../css/component-usage.css" type="text/css" />
</head>
<body>
<!-- Processor Documentation ================================================== -->
<h2>Description:</h2>
<p>
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.
<p>
<strong>Properties:</strong>
</p>
<p>
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.
</p>
<ul>
<li><strong>Match Requirement</strong>
<ul>
<li>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:
<ul>
<li>content must match exactly</li>
<li>content must contain match</li>
</ul></li>
<li>Default value: content must match exactly</li>
<li>Supports expression language: false</li>
</ul></li>
<li><strong>Character Set</strong>
<ul>
<li>The character set in which the file is encoded.</li>
<li>Default value: UTF-8</li>
<li>Supports expression language: false</li>
</ul></li>
<li><strong>Content Buffer Size</strong>
<ul>
<li>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. </li>
<li>Default value: 1 MB</li>
<li>Supports expression language: false</li>
</ul></li>
<li>User-Defined Properties
<ul>
<li>Users add properties with regular expressions (see the description above).</li>
<li>User-defined properties do support the NiFi Expression Language, but in such cases, the results are interpreted as literal values, not regular expressions.</li>
</ul></li>
</ul>
<p>
<strong>Relationships:</strong>
</p>
<ul>
<li>unmatched
<ul>
<li>FlowFiles that do not match any user-defined criteria follow this relationship.</li>
</ul></li>
<li>user-defined relationships
<ul>
<li>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).</li>
</ul></li>
</ul>
</body>
</html>

View File

@ -1,85 +0,0 @@
<!DOCTYPE html>
<html lang="en">
<!--
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.
-->
<head>
<meta charset="utf-8" />
<title>ScanAttribute</title>
<link rel="stylesheet" href="../../css/component-usage.css" type="text/css" />
</head>
<body>
<!-- Processor Documentation ================================================== -->
<h2>Description:</h2>
<p>
This processor scans the specified attributes of FlowFiles, checking to see if any of their values are present within the specified dictionary of terms.
<p>
<strong>Properties:</strong>
</p>
<p>
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.
</p>
<ul>
<li><strong>Dictionary File</strong>
<ul>
<li>The local file path to a new-line-delimited text file that includes terms that should trigger a match. Empty lines are ignored.</li>
<li>Default value: no default</li>
<li>Supports expression language: false</li>
</ul></li>
<li><strong>Attribute Pattern</strong>
<ul>
<li>A Java regular expression that specifies the names of attributes whose values will be matched against the terms in the dictionary file.</li>
<li>Default value: .*</li>
<li>Supports expression language: false</li>
</ul></li>
<li><strong>Match Criteria</strong>
<ul>
<li>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. </li>
<li>Default value: At Least 1 Must Match</li>
<li>Supports expression language: false</li>
</ul></li>
<!--
<li><strong>Attribute Value Match Criteria</strong>
<ul>
<li>If set to "All of the Value", a dictionary value needs to match the complete attribute value. If set to "Part of the Value", the dictionary value match part of the attribute value. </li>
<li>Default value: Match All of the Value</li>
<li>Supports expression language: false</li>
</ul></li>
-->
<li>Dictionary Filter Pattern
<ul>
<li>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.</li>
<li>Supports expression language: false</li>
</ul></li>
</ul>
<p>
<strong>Relationships:</strong>
</p>
<ul>
<li>matched
<ul>
<li>FlowFiles whose attributes are found in the dictionary file follow this relationship.</li>
</ul></li>
<li>unmatched
<ul>
<li>FlowFiles whose attributes are not found in the dictionary file follow this relationship.</li>
</ul></li>
</ul>
</body>
</html>

View File

@ -0,0 +1,45 @@
<!DOCTYPE html>
<html lang="en">
<!--
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.
-->
<head>
<meta charset="utf-8" />
<title>ScanContent</title>
<link rel="stylesheet" href="../../css/component-usage.css" type="text/css" />
</head>
<body>
<!-- Processor Documentation ================================================== -->
<p>
<strong>Modifies Attributes:</strong>
</p>
<table>
<thead>
<tr>
<th>Attribute Name</th>
<th>Description</th>
</tr>
</thead>
<tbody>
<tr>
<td>matching.term</td>
<td>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.</td>
</tr>
</tbody>
</table>
</body>
</html>

View File

@ -1,100 +0,0 @@
<!DOCTYPE html>
<html lang="en">
<!--
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.
-->
<head>
<meta charset="utf-8" />
<title>ScanContent</title>
<link rel="stylesheet" href="../../css/component-usage.css" type="text/css" />
</head>
<body>
<!-- Processor Documentation ================================================== -->
<h2>Description:</h2>
<p>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.
</p>
<p>
<strong>Modifies Attributes:</strong>
</p>
<table border="1">
<thead>
<tr>
<th>Attribute Name</th>
<th>Description</th>
</tr>
</thead>
<tbody>
<tr>
<td>matching.term</td>
<td>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.</td>
</tr>
</tbody>
</table>
<p>
<strong>Properties:</strong>
</p>
<p>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.</p>
<ul>
<li><strong>Dictionary File</strong>
<ul>
<li>The local file path and filename for the dictionary file.</li>
<li>Default value: no default</li>
<li>Supports expression language: false</li>
</ul></li>
<li><strong>Dictionary Encoding</strong>
<ul>
<li>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.</li>
<li>Default value: text</li>
<li>Supports expression language: false</li>
</ul></li>
</ul>
<p>
<strong>Relationships:</strong>
</p>
<ul>
<li>matched
<ul>
<li>If FlowFiles match at least one term in the dictionary
file, then they follow this relationship.</li>
</ul>
</li>
<li>unmatched
<ul>
<li>If FlowFiles do not match any term in the dictionary file,
then they follow this relationship.</li>
</ul>
</li>
</ul>
</body>
</html>

View File

@ -23,14 +23,11 @@
<body> <body>
<!-- Processor Documentation ================================================== --> <!-- Processor Documentation ================================================== -->
<h2>Description:</h2>
<p>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.</p>
<p> <p>
<strong>Adds or Modifies Attributes:</strong> <strong>Adds or Modifies Attributes:</strong>
</p> </p>
<table border="1"> <table>
<thead> <thead>
<tr> <tr>
<th>Attribute Name</th> <th>Attribute Name</th>
@ -74,44 +71,6 @@
</tr> </tr>
</tbody> </tbody>
</table> </table>
<p>
<strong>Properties:</strong>
</p>
<p>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.</p>
<ul>
<li><strong>Segment Size</strong>
<ul>
<li>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.</li>
<li>Default value: no default</li>
<li>Supports expression language: false</li>
</ul></li>
</ul>
<p>
<strong>Relationships:</strong>
</p>
<ul>
<li>original
<ul>
<li>The original FlowFile will be sent to this relationship.</li>
</ul>
</li>
<li>segments
<ul>
<li>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.</li>
</ul>
</li>
</ul>
<p> <p>
<strong>See Also:</strong> <strong>See Also:</strong>
</p> </p>

View File

@ -23,15 +23,11 @@
<body> <body>
<!-- Processor Documentation ================================================== --> <!-- Processor Documentation ================================================== -->
<h2>Description:</h2>
<p>
This processor splits incoming FlowFiles by a specified byte sequence.
</p>
<p> <p>
<strong>Modifies Attributes:</strong> <strong>Modifies Attributes:</strong>
</p> </p>
<table border="1"> <table>
<thead> <thead>
<tr> <tr>
<th>Attribute Name</th> <th>Attribute Name</th>
@ -60,42 +56,6 @@
</tr> </tr>
</tbody> </tbody>
</table> </table>
<p>
<strong>Properties:</strong>
</p>
<p>
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.
</p>
<ul>
<li><strong>Byte Sequence</strong>
<ul>
<li>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.</li>
<li>Default value: no default</li>
<li>Supports expression language: false</li>
</ul></li>
<li><strong>Keep Byte Sequence</strong>
<ul>
<li>A Boolean value (true/false), indicating whether the byte sequence should be included at the end of each split. </li>
<li>Default value: false</li>
<li>Supports expression language: false</li>
</ul></li>
</ul>
<p>
<strong>Relationships:</strong>
</p>
<ul>
<li>original
<ul>
<li>A copy of the original FlowFile follows this relationship.</li>
</ul></li>
<li>splits
<ul>
<li>The resulting split files that are produced by this processor follow this relationship.</li>
</ul></li>
</ul>
<p> <p>
<strong>See Also:</strong> <strong>See Also:</strong>
</p> </p>

View File

@ -23,15 +23,11 @@
<body> <body>
<!-- Processor Documentation ================================================== --> <!-- Processor Documentation ================================================== -->
<h2>Description:</h2>
<p>
This processor splits a text file into multiple smaller text files on line boundaries, each having up to a configured number of lines.
</p>
<p> <p>
<strong>Modifies Attributes:</strong> <strong>Modifies Attributes:</strong>
</p> </p>
<table border="1"> <table>
<thead> <thead>
<tr> <tr>
<th>Attribute Name</th> <th>Attribute Name</th>
@ -65,51 +61,6 @@
</tbody> </tbody>
</table> </table>
<p>
<strong>Properties:</strong>
</p>
<p>
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.
</p>
<ul>
<li><strong>Line Split Count</strong>
<ul>
<li>The number of lines to be included in each split file. The value must be a non-negative integer.</li>
<li>Default value: no default</li>
<li>Supports expression language: false</li>
</ul></li>
<li><strong>Header Line Count</strong>
<ul>
<li>The number of lines that should be considered part of a header on each split file.</li>
<li>Default value: 0</li>
<li>Supports expression language: false</li>
</ul></li>
<li><strong>Remove Trailing Newlines</strong>
<ul>
<li>Whether to remove newlines at the end of each split file. This should be false if you intend to merge the split files later.</li>
<li>Default value: true</li>
<li>Supports expression language: false</li>
</ul></li>
</ul>
<p>
<strong>Relationships:</strong>
</p>
<ul>
<li>failure
<ul>
<li>If FlowFiles cannot be split for some reason, the original file follows this relationship and does not follow the original relationship.</li>
</ul></li>
<li>original
<ul>
<li>If FlowFiles are successfully split into one or more files, a copy of the original file follows this relationship.</li>
</ul></li>
<li>splits
<ul>
<li>If FlowFiles are successfully split into one or more files, those split files follow this relationship.</li>
</ul></li>
</ul>
<p> <p>
<strong>See Also:</strong> <strong>See Also:</strong>
</p> </p>

View File

@ -1,64 +0,0 @@
<!DOCTYPE html>
<html lang="en">
<!--
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.
-->
<head>
<meta charset="utf-8" />
<title>SplitXML</title>
<link rel="stylesheet" href="../../css/component-usage.css" type="text/css" />
</head>
<body>
<!-- Processor Documentation ================================================== -->
<h2>Description:</h2>
<p>
This processor splits an XML file into multiple separate FlowFiles, each comprising a child or descendant of the original root element.
<p>
<strong>Properties:</strong>
</p>
<p>
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.
</p>
<ul>
<li><strong>Line Depth</strong>
<ul>
<li>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.</li>
<li>Default value: 1</li>
<li>Supports expression language: false</li>
</ul></li>
</ul>
<p>
<strong>Relationships:</strong>
</p>
<ul>
<li>failure
<ul>
<li>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.</li>
</ul></li>
<li>original
<ul>
<li>If FlowFiles are successfully split, a copy of the original FlowFile follows this relationship.</li>
</ul></li>
<li>splits
<ul>
<li>If FlowFiles are successfully split into one or more files, those split files follow this relationship.</li>
</ul></li>
</ul>
</body>
</html>

Some files were not shown because too many files have changed in this diff Show More