mirror of https://github.com/apache/nifi.git
NIFI-3027: ExecuteProcess supports the expression language
This commit is contained in:
parent
ed6e03399f
commit
91f0170416
|
@ -82,7 +82,7 @@ public class ExecuteProcess extends AbstractProcessor {
|
||||||
.name("Command Arguments")
|
.name("Command Arguments")
|
||||||
.description("The arguments to supply to the executable delimited by white space. White space can be escaped by enclosing it in double-quotes.")
|
.description("The arguments to supply to the executable delimited by white space. White space can be escaped by enclosing it in double-quotes.")
|
||||||
.required(false)
|
.required(false)
|
||||||
.expressionLanguageSupported(false)
|
.expressionLanguageSupported(true)
|
||||||
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
|
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
|
||||||
.build();
|
.build();
|
||||||
|
|
||||||
|
@ -276,8 +276,10 @@ public class ExecuteProcess extends AbstractProcessor {
|
||||||
|
|
||||||
protected List<String> createCommandStrings(final ProcessContext context) {
|
protected List<String> createCommandStrings(final ProcessContext context) {
|
||||||
final String command = context.getProperty(COMMAND).getValue();
|
final String command = context.getProperty(COMMAND).getValue();
|
||||||
final List<String> args = ArgumentUtils.splitArgs(context.getProperty(COMMAND_ARGUMENTS).getValue(),
|
final String arguments = context.getProperty(COMMAND_ARGUMENTS).isSet()
|
||||||
context.getProperty(ARG_DELIMITER).getValue().charAt(0));
|
? context.getProperty(COMMAND_ARGUMENTS).evaluateAttributeExpressions().getValue()
|
||||||
|
: null;
|
||||||
|
final List<String> args = ArgumentUtils.splitArgs(arguments, context.getProperty(ARG_DELIMITER).getValue().charAt(0));
|
||||||
|
|
||||||
final List<String> commandStrings = new ArrayList<>(args.size() + 1);
|
final List<String> commandStrings = new ArrayList<>(args.size() + 1);
|
||||||
commandStrings.add(command);
|
commandStrings.add(command);
|
||||||
|
|
|
@ -268,6 +268,31 @@ public class TestExecuteProcess {
|
||||||
assertEquals(1, succeeded.size());
|
assertEquals(1, succeeded.size());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testRedirectErrorStreamWithExpressions() {
|
||||||
|
final TestRunner runner = TestRunners.newTestRunner(ExecuteProcess.class);
|
||||||
|
runner.setProperty(ExecuteProcess.COMMAND, "ls");
|
||||||
|
runner.setProperty(ExecuteProcess.COMMAND_ARGUMENTS, "${literal('does-not-exist'):toUpper()}");
|
||||||
|
runner.setProperty(ExecuteProcess.REDIRECT_ERROR_STREAM, "true");
|
||||||
|
|
||||||
|
ProcessContext processContext = runner.getProcessContext();
|
||||||
|
|
||||||
|
ExecuteProcess processor = (ExecuteProcess) runner.getProcessor();
|
||||||
|
processor.updateScheduledTrue();
|
||||||
|
processor.setupExecutor(processContext);
|
||||||
|
|
||||||
|
processor.onTrigger(processContext, runner.getProcessSessionFactory());
|
||||||
|
|
||||||
|
if (isCommandFailed(runner)) return;
|
||||||
|
|
||||||
|
final List<LogMessage> warnMessages = runner.getLogger().getWarnMessages();
|
||||||
|
assertEquals("If redirect error stream is true " +
|
||||||
|
"the output should be sent as a content of flow-file.", 0, warnMessages.size());
|
||||||
|
final List<MockFlowFile> succeeded = runner.getFlowFilesForRelationship(ExecuteProcess.REL_SUCCESS);
|
||||||
|
assertEquals(1, succeeded.size());
|
||||||
|
assertTrue(new String(succeeded.get(0).toByteArray()).contains("DOES-NOT-EXIST"));
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* On some environment, the test command immediately fail with an IOException
|
* On some environment, the test command immediately fail with an IOException
|
||||||
* because of the native UnixProcess.init method implementation difference.
|
* because of the native UnixProcess.init method implementation difference.
|
||||||
|
|
Loading…
Reference in New Issue