NIFI-380 fixed unit test to use a more os portable command.  Modify execute 
process to enable error stream redirection.


Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/31211a70
Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/31211a70
Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/31211a70

Branch: refs/heads/NIFI-380
Commit: 31211a70a786bf903c12744c90e673cb683cd112
Parents: bfff6dc
Author: joewitt <joew...@apache.org>
Authored: Thu Mar 5 22:56:41 2015 -0500
Committer: joewitt <joew...@apache.org>
Committed: Thu Mar 5 22:56:41 2015 -0500

----------------------------------------------------------------------
 .../processors/standard/ExecuteProcess.java     | 272 ++++++++++---------
 .../index.html                                  |  42 +--
 .../processors/standard/TestExecuteProcess.java |   6 +-
 3 files changed, 168 insertions(+), 152 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/31211a70/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ExecuteProcess.java
----------------------------------------------------------------------
diff --git 
a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ExecuteProcess.java
 
b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ExecuteProcess.java
index c510641..31efd18 100644
--- 
a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ExecuteProcess.java
+++ 
b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ExecuteProcess.java
@@ -38,7 +38,6 @@ import java.util.concurrent.Future;
 import java.util.concurrent.ThreadFactory;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicReference;
 import java.util.concurrent.locks.Lock;
 import java.util.concurrent.locks.ReentrantLock;
 
@@ -58,7 +57,6 @@ import org.apache.nifi.processor.exception.ProcessException;
 import org.apache.nifi.processor.io.OutputStreamCallback;
 import org.apache.nifi.processor.util.StandardValidators;
 
-
 @Tags({"command", "process", "source", "external", "invoke", "script"})
 @CapabilityDescription("Runs an operating system command specified by the user 
and writes the output of that command to a FlowFile. If the command is expected 
"
         + "to be long-running, the Processor can output the partial data on a 
specified interval. When this option is used, the output is expected to be in 
textual "
@@ -66,94 +64,105 @@ import org.apache.nifi.processor.util.StandardValidators;
 public class ExecuteProcess extends AbstractProcessor {
 
     public static final PropertyDescriptor COMMAND = new 
PropertyDescriptor.Builder()
-        .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)
-        .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-        .build();
-    
+            .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)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .build();
+
     public static final PropertyDescriptor COMMAND_ARGUMENTS = new 
PropertyDescriptor.Builder()
-        .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.")
-        .required(false)
-        .expressionLanguageSupported(false)
-        .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-        .build();
+            .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.")
+            .required(false)
+            .expressionLanguageSupported(false)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .build();
 
     public static final PropertyDescriptor WORKING_DIR = new 
PropertyDescriptor.Builder()
-        .name("Working Directory")
-        .description("The directory to use as the current working directory 
when executing the command")
-        .expressionLanguageSupported(false)
-        .addValidator(StandardValidators.createDirectoryExistsValidator(false, 
true))
-        .required(false)
-        .build();
-    
+            .name("Working Directory")
+            .description("The directory to use as the current working 
directory when executing the command")
+            .expressionLanguageSupported(false)
+            
.addValidator(StandardValidators.createDirectoryExistsValidator(false, true))
+            .required(false)
+            .build();
+
     public static final PropertyDescriptor BATCH_DURATION = new 
PropertyDescriptor.Builder()
-        .name("Batch Duration")
-        .description("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")
-        .required(false)
-        .expressionLanguageSupported(false)
-        .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
-        .build();
-    
+            .name("Batch Duration")
+            .description("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")
+            .required(false)
+            .expressionLanguageSupported(false)
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .build();
+
+    public static final PropertyDescriptor REDIRECT_ERROR_STREAM = new 
PropertyDescriptor.Builder()
+            .name("Redirect Error Stream")
+            .description("If true will redirect any error stream output of the 
process to the output stream. "
+                    + "This is particularly helpful for processes which write 
extensively to the error stream or for troubleshooting.")
+            .required(false)
+            .allowableValues("true", "false")
+            .defaultValue("false")
+            .expressionLanguageSupported(false)
+            .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
+            .build();
+
     public static final Relationship REL_SUCCESS = new Relationship.Builder()
-        .name("success")
-        .description("All created FlowFiles are routed to this relationship")
-        .build();
-    
+            .name("success")
+            .description("All created FlowFiles are routed to this 
relationship")
+            .build();
 
     private volatile ExecutorService executor;
-    
+
     @Override
     public Set<Relationship> getRelationships() {
         return Collections.singleton(REL_SUCCESS);
     }
-    
+
     @Override
     protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
         final List<PropertyDescriptor> properties = new ArrayList<>();
         properties.add(COMMAND);
         properties.add(COMMAND_ARGUMENTS);
         properties.add(BATCH_DURATION);
+        properties.add(REDIRECT_ERROR_STREAM);
         return properties;
     }
-    
+
     @Override
     protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final 
String propertyDescriptorName) {
         return new PropertyDescriptor.Builder()
-            .name(propertyDescriptorName)
-            .description("Sets the environment variable '" + 
propertyDescriptorName + "' for the process' environment")
-            .dynamic(true)
-            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
-            .build();
+                .name(propertyDescriptorName)
+                .description("Sets the environment variable '" + 
propertyDescriptorName + "' for the process' environment")
+                .dynamic(true)
+                .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+                .build();
     }
-    
+
     static List<String> splitArgs(final String input) {
-        if ( input == null ) {
+        if (input == null) {
             return Collections.emptyList();
         }
-        
+
         final List<String> args = new ArrayList<>();
-        
+
         final String trimmed = input.trim();
         boolean inQuotes = false;
-        
+
         final StringBuilder sb = new StringBuilder();
-        for (int i=0; i < trimmed.length(); i++) {
+        for (int i = 0; i < trimmed.length(); i++) {
             final char c = trimmed.charAt(i);
             switch (c) {
                 case ' ':
                 case '\t':
                 case '\r':
                 case '\n': {
-                    if ( inQuotes ) {
+                    if (inQuotes) {
                         sb.append(c);
                     } else {
                         final String arg = sb.toString().trim();
-                        if ( !arg.isEmpty() ) {
+                        if (!arg.isEmpty()) {
                             args.add(arg);
                         }
                         sb.setLength(0);
@@ -168,20 +177,20 @@ public class ExecuteProcess extends AbstractProcessor {
                     break;
             }
         }
-        
+
         final String finalArg = sb.toString().trim();
-        if ( !finalArg.isEmpty() ) {
+        if (!finalArg.isEmpty()) {
             args.add(finalArg);
         }
-        
+
         return args;
     }
-    
+
     @OnScheduled
     public void setupExecutor(final ProcessContext context) {
         executor = 
Executors.newFixedThreadPool(context.getMaxConcurrentTasks() * 2, new 
ThreadFactory() {
             private final ThreadFactory defaultFactory = 
Executors.defaultThreadFactory();
-            
+
             @Override
             public Thread newThread(final Runnable r) {
                 final Thread t = defaultFactory.newThread(r);
@@ -190,73 +199,67 @@ public class ExecuteProcess extends AbstractProcessor {
             }
         });
     }
-    
+
     @OnUnscheduled
     public void shutdownExecutor() {
         executor.shutdown();
     }
-    
+
     @Override
     public void onTrigger(final ProcessContext context, final ProcessSession 
session) throws ProcessException {
         final String command = context.getProperty(COMMAND).getValue();
         final List<String> args = 
splitArgs(context.getProperty(COMMAND_ARGUMENTS).getValue());
-        
+        final Boolean redirectErrorStream = 
context.getProperty(REDIRECT_ERROR_STREAM).asBoolean();
+
         final List<String> commandStrings = new ArrayList<>(args.size() + 1);
         commandStrings.add(command);
         commandStrings.addAll(args);
 
         final String commandString = StringUtils.join(commandStrings, " ");
-        
+
         final ProcessBuilder builder = new ProcessBuilder(commandStrings);
         final String workingDirName = 
context.getProperty(WORKING_DIR).getValue();
-        if ( workingDirName != null ) {
+        if (workingDirName != null) {
             builder.directory(new File(workingDirName));
         }
-        
+
         final Map<String, String> environment = new HashMap<>();
-        for ( final Map.Entry<PropertyDescriptor, String> entry : 
context.getProperties().entrySet() ) {
-            if ( entry.getKey().isDynamic() ) {
+        for (final Map.Entry<PropertyDescriptor, String> entry : 
context.getProperties().entrySet()) {
+            if (entry.getKey().isDynamic()) {
                 environment.put(entry.getKey().getName(), entry.getValue());
             }
         }
-        
-        if ( !environment.isEmpty() ) {
+
+        if (!environment.isEmpty()) {
             builder.environment().putAll(environment);
         }
-        
+
         final long startNanos = System.nanoTime();
         final Process process;
         try {
-            process = builder.start();
+            process = builder.redirectErrorStream(redirectErrorStream).start();
         } catch (final IOException ioe) {
-            getLogger().error("Failed to create process due to {}", new 
Object[] {ioe});
+            getLogger().error("Failed to create process due to {}", new 
Object[]{ioe});
             context.yield();
             return;
         }
-        
+
         final Long batchNanos = 
context.getProperty(BATCH_DURATION).asTimePeriod(TimeUnit.NANOSECONDS);
-        
+
         // Submit task to read error stream from process
-        final AtomicReference<String> errorStream = new AtomicReference<>();
-        executor.submit(new Runnable() {
-            @Override
-            public void run() {
-                final StringBuilder sb = new StringBuilder();
-                try (final BufferedReader reader = new BufferedReader(new 
InputStreamReader(process.getErrorStream()))) {
-                    String line;
-                    while ((line = reader.readLine()) != null) {
-                        if ( sb.length() < 4000 ) {
-                            sb.append(line);
-                            sb.append("\n");
+        if (!redirectErrorStream) {
+            executor.submit(new Runnable() {
+                @Override
+                public void run() {
+                    try (final BufferedReader reader = new BufferedReader(new 
InputStreamReader(process.getErrorStream()))) {
+                        while (reader.read() >= 0) {
                         }
+                    } catch (final IOException ioe) {
                     }
-                } catch (final IOException ioe) {
                 }
-                
-                errorStream.set(sb.toString());
-            }
-        });
-        
+            });
+        }
+
         // Submit task to read output of Process and write to FlowFile.
         final ProxyOutputStream proxyOut = new ProxyOutputStream(getLogger());
         final AtomicBoolean failure = new AtomicBoolean(false);
@@ -265,16 +268,16 @@ public class ExecuteProcess extends AbstractProcessor {
             @Override
             public Object call() throws IOException {
                 try {
-                    if ( batchNanos == null ) {
+                    if (batchNanos == null) {
                         // if we aren't batching, just copy the stream from 
the process to the flowfile.
                         try (final BufferedInputStream bufferedIn = new 
BufferedInputStream(process.getInputStream())) {
                             final byte[] buffer = new byte[4096];
                             int len;
                             while ((len = bufferedIn.read(buffer)) > 0) {
-                                if ( !isScheduled() ) {
+                                if (!isScheduled()) {
                                     return null;
                                 }
-                                
+
                                 proxyOut.write(buffer, 0, len);
                             }
                         }
@@ -286,9 +289,9 @@ public class ExecuteProcess extends AbstractProcessor {
                         // to read lines of text and write them as lines of 
text.
                         try (final BufferedReader reader = new 
BufferedReader(new InputStreamReader(process.getInputStream()))) {
                             String line;
-                            
+
                             while ((line = reader.readLine()) != null) {
-                                if ( !isScheduled() ) {
+                                if (!isScheduled()) {
                                     return null;
                                 }
 
@@ -302,22 +305,22 @@ public class ExecuteProcess extends AbstractProcessor {
                 } finally {
                     finishedCopying.set(true);
                 }
-                
+
                 return null;
             }
         });
-        
+
         // continue to do this loop until both the process has finished and we 
have finished copying
         // the output from the process to the FlowFile. Unfortunately, even 
after calling Process.exitValue(), 
         // there can be data buffered on the InputStream; so we will wait 
until the stream is empty as well.
         int flowFileCount = 0;
         while (!finishedCopying.get() || isAlive(process)) {
-            if ( !isScheduled() ) {
+            if (!isScheduled()) {
                 getLogger().info("User stopped processor; will terminate 
process immediately");
                 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
@@ -327,31 +330,33 @@ public class ExecuteProcess extends AbstractProcessor {
                 public void process(final OutputStream flowFileOut) throws 
IOException {
                     try (final OutputStream out = new 
BufferedOutputStream(flowFileOut)) {
                         proxyOut.setDelegate(out);
-                        
-                        if ( batchNanos == null ) {
+
+                        if (batchNanos == null) {
                             // we are not creating batches; wait until process 
terminates.
                             Integer exitCode = null;
                             while (exitCode == null) {
                                 try {
                                     exitCode = process.waitFor();
-                                } catch (final InterruptedException ie) {}
+                                } catch (final InterruptedException ie) {
+                                }
                             }
                         } else {
                             // wait the allotted amount of time.
                             try {
                                 TimeUnit.NANOSECONDS.sleep(batchNanos);
-                            } catch (final InterruptedException ie) {}
+                            } catch (final InterruptedException ie) {
+                            }
                         }
-                        
+
                         proxyOut.setDelegate(null); // prevent from writing to 
this stream
                     }
                 }
             });
-            
-            if ( flowFile.getSize() == 0L ) {
+
+            if (flowFile.getSize() == 0L) {
                 // If no data was written to the file, remove it
                 session.remove(flowFile);
-            } else if ( failure.get() ) {
+            } 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");
@@ -359,15 +364,15 @@ public class ExecuteProcess extends AbstractProcessor {
             } 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});
+                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();
         }
-        
+
         final int exitCode;
         final long millis = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - 
startNanos);
         try {
@@ -376,20 +381,19 @@ public class ExecuteProcess extends AbstractProcessor {
             getLogger().warn("Process was interrupted before finishing");
             return;
         }
-                
+
         try {
             future.get();
         } catch (final ExecutionException e) {
-            getLogger().error("Failed to copy output from Process to FlowFile 
due to {}", new Object[] {e.getCause()});
+            getLogger().error("Failed to copy output from Process to FlowFile 
due to {}", new Object[]{e.getCause()});
         } catch (final InterruptedException ie) {
             getLogger().error("Interrupted while waiting to copy data form 
Process to FlowFile");
             return;
         }
-        
-        getLogger().info("Process finished with exit code {} after creating {} 
FlowFiles in {} millis", new Object[] {exitCode, flowFileCount, millis});
+
+        getLogger().info("Process finished with exit code {} after creating {} 
FlowFiles in {} millis", new Object[]{exitCode, flowFileCount, millis});
     }
 
-    
     private boolean isAlive(final Process process) {
         // unfortunately, java provides no straight-forward way to test if a 
Process is alive.
         // In Java 8, Process.isAlive() is introduced, but NiFi needs to run 
against Java 7,
@@ -401,46 +405,48 @@ public class ExecuteProcess extends AbstractProcessor {
             return true;
         }
     }
-    
-    
+
     /**
-     * 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
+     * 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;
-        
+
         private final Lock lock = new ReentrantLock();
         private OutputStream delegate;
 
         public ProxyOutputStream(final ProcessorLog logger) {
             this.logger = logger;
         }
-        
+
         public void setDelegate(final OutputStream delegate) {
             lock.lock();
             try {
-                logger.trace("Switching delegate from {} to {}", new Object[] 
{this.delegate, delegate});
+                logger.trace("Switching delegate from {} to {}", new 
Object[]{this.delegate, delegate});
 
                 this.delegate = delegate;
             } finally {
                 lock.unlock();
             }
         }
-        
+
         private void sleep(final long millis) {
             try {
                 Thread.sleep(millis);
-            } catch (final InterruptedException ie) {}
+            } catch (final InterruptedException ie) {
+            }
         }
-        
+
         @Override
         public void write(final int b) throws IOException {
             lock.lock();
             try {
                 while (true) {
-                    if ( delegate != null ) {
-                        logger.trace("Writing to {}", new Object[] {delegate});
+                    if (delegate != null) {
+                        logger.trace("Writing to {}", new Object[]{delegate});
 
                         delegate.write(b);
                         return;
@@ -454,15 +460,15 @@ public class ExecuteProcess extends AbstractProcessor {
                 lock.unlock();
             }
         }
-        
+
         @Override
         public void write(final byte[] b, final int off, final int len) throws 
IOException {
             lock.lock();
             try {
                 while (true) {
-                    if ( delegate != null ) {
-                        logger.trace("Writing to {}", new Object[] {delegate});
-                        
+                    if (delegate != null) {
+                        logger.trace("Writing to {}", new Object[]{delegate});
+
                         delegate.write(b, off, len);
                         return;
                     } else {
@@ -475,22 +481,22 @@ public class ExecuteProcess extends AbstractProcessor {
                 lock.unlock();
             }
         }
-        
+
         @Override
         public void write(final byte[] b) throws IOException {
             write(b, 0, b.length);
         }
-        
+
         @Override
         public void close() throws IOException {
         }
-        
+
         @Override
         public void flush() throws IOException {
             lock.lock();
             try {
                 while (true) {
-                    if ( delegate != null ) {
+                    if (delegate != null) {
                         delegate.flush();
                         return;
                     } else {

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/31211a70/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.ExecuteProcess/index.html
----------------------------------------------------------------------
diff --git 
a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.ExecuteProcess/index.html
 
b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.ExecuteProcess/index.html
index 9129855..3526cd1 100644
--- 
a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.ExecuteProcess/index.html
+++ 
b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.ExecuteProcess/index.html
@@ -25,11 +25,11 @@
         <!-- 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
+            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>
@@ -60,20 +60,30 @@
                     <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>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>
+            </li>
+            <li>Redirect Error Stream
+                <ul>
+                    <li>
+                        If true will redirect any error stream output of the 
process to the output stream.
+                        This is particularly helpful for processes which write 
extensively to the error stream or for troubleshooting.
+                    </li>
+                    <li>Default value: false</li>
+                    <li>Allowed Values: true, false</li>
+                    <li>Supports expression language: false</li>
+                </ul>
+            </li>
         </ul>
 
         <p>

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/31211a70/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestExecuteProcess.java
----------------------------------------------------------------------
diff --git 
a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestExecuteProcess.java
 
b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestExecuteProcess.java
index 0aa2ee3..897973c 100644
--- 
a/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestExecuteProcess.java
+++ 
b/nifi/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestExecuteProcess.java
@@ -59,12 +59,12 @@ public class TestExecuteProcess {
     }
     
     @Test
-    public void testPing() {
+    public void testEcho() {
         System.setProperty("org.slf4j.simpleLogger.log.org.apache.nifi", 
"TRACE");
         
         final TestRunner runner = 
TestRunners.newTestRunner(ExecuteProcess.class);
-        runner.setProperty(ExecuteProcess.COMMAND, "ping");
-        runner.setProperty(ExecuteProcess.COMMAND_ARGUMENTS, "127.0.0.1");
+        runner.setProperty(ExecuteProcess.COMMAND, "echo");
+        runner.setProperty(ExecuteProcess.COMMAND_ARGUMENTS, "test-args");
         runner.setProperty(ExecuteProcess.BATCH_DURATION, "500 millis");
         
         runner.run();

Reply via email to