NIFI-380: Added documentation

Signed-off-by: joewitt <joewitt@apache.org>
This commit is contained in:
Mark Payne 2015-03-04 20:09:08 -05:00 committed by joewitt
parent e4ab0f4a76
commit bd066cd567
2 changed files with 110 additions and 7 deletions

View File

@ -57,7 +57,6 @@ import org.apache.nifi.processor.Relationship;
import org.apache.nifi.processor.exception.ProcessException;
import org.apache.nifi.processor.io.OutputStreamCallback;
import org.apache.nifi.processor.util.StandardValidators;
import org.apache.nifi.stream.io.StreamUtils;
@Tags({"command", "process", "source", "external", "invoke", "script"})
@ -66,13 +65,8 @@ import org.apache.nifi.stream.io.StreamUtils;
+ "format, as it typically does not make sense to split binary data on arbitrary time-based intervals.")
public class ExecuteProcess extends AbstractProcessor {
public static final Relationship REL_SUCCESS = new Relationship.Builder()
.name("success")
.description("All created FlowFiles are routed to this relationship")
.build();
public static final PropertyDescriptor COMMAND = new PropertyDescriptor.Builder()
.name("Command Path")
.name("Command")
.description("Specifies the command to be executed; if just the name of an executable is provided, it must be in the user's environment PATH.")
.required(true)
.expressionLanguageSupported(false)
@ -105,6 +99,12 @@ public class ExecuteProcess extends AbstractProcessor {
.addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
.build();
public static final Relationship REL_SUCCESS = new Relationship.Builder()
.name("success")
.description("All created FlowFiles are routed to this relationship")
.build();
private volatile ExecutorService executor;
@Override
@ -317,6 +317,10 @@ public class ExecuteProcess extends AbstractProcessor {
process.destroy();
break;
}
// Create a FlowFile that we can write to and set the OutputStream for the FlowFile
// as the delegate for the ProxyOuptutStream, then wait until the process finishes
// or until the specified amount of time
FlowFile flowFile = session.create();
flowFile = session.write(flowFile, new OutputStreamCallback() {
@Override
@ -345,17 +349,22 @@ public class ExecuteProcess extends AbstractProcessor {
});
if ( flowFile.getSize() == 0L ) {
// If no data was written to the file, remove it
session.remove(flowFile);
} else if ( failure.get() ) {
// If there was a failure processing the output of the Process, remove the FlowFile
session.remove(flowFile);
getLogger().error("Failed to read data from Process, so will not generate FlowFile");
break;
} else {
// All was good. Generate event and transfer FlowFile.
session.getProvenanceReporter().create(flowFile, "Created from command: " + commandString);
getLogger().info("Created {} and routed to success", new Object[] {flowFile});
session.transfer(flowFile, REL_SUCCESS);
flowFileCount++;
}
// Commit the session so that the FlowFile is transferred to the next processor
session.commit();
}
@ -394,6 +403,10 @@ public class ExecuteProcess extends AbstractProcessor {
}
/**
* Output stream that is used to wrap another output stream in a way that the
* underlying output stream can be swapped out for a different one when needed
*/
private static class ProxyOutputStream extends OutputStream {
private final ProcessorLog logger;

View File

@ -0,0 +1,90 @@
<!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>ExecuteProcess</title>
<link rel="stylesheet" href="../../css/component-usage.css" type="text/css" />
</head>
<body>
<!-- Processor Documentation ================================================== -->
<h2> Description:</h2>
<p>
Runs an operating system command specified by the user and writes the output of that command to a FlowFile. If the command is expected
to be long-running, the Processor can output the partial data on a specified interval. When this option is used, the output is expected to be in textual
format, as it typically does not make sense to split binary data on arbitrary time-based intervals.
</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>Command</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: false</li>
</ul>
</li>
<li>Command Arguments
<ul>
<li>The arguments to supply to the executable delimited by white space. White space can be escaped by enclosing it in double-quotes.</li>
<li>Default value: none</li>
<li>Supports expression language: false</li>
</ul>
</li>
<li>Working Directory
<ul>
<li>The directory to use as the current working directory when executing the command</li>
<li>Default value: none (which means whatever NiFi's root installation directory is)</li>
<li>Supports expression language: false</li>
</ul>
</li>
<li>
Batch Duration>
<ul>
<li>
If the process is expected to be long-running and produce textual output, a batch duration can be specified so
that the output will be captured for this amount of time and a FlowFile will then be sent out with the results
and a new FlowFile will be started, rather than waiting for the process to finish before sending out the results.
If no value is provided, the process will run to completion and the entire output of the process will be written
to a single FlowFile.
</li>
<li>Default value: none</li>
<li>Supports expression language: false</li>
</ul>
</li>
</ul>
<p>
<strong>Relationships:</strong>
</p>
<ul>
<li>success
<ul>
<li>All FlowFiles that are created are routed to this relationship.</li>
</ul>
</li>
</ul>
</body>
</html>