You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@nifi.apache.org by jo...@apache.org on 2014/12/08 21:29:54 UTC

[31/51] [partial] incubator-nifi git commit: Initial code contribution

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/nar-bundles/execute-script-bundle/execute-script-processors/src/main/resources/docs/org.apache.nifi.processors.script.ExecuteScript/index.html
----------------------------------------------------------------------
diff --git a/nar-bundles/execute-script-bundle/execute-script-processors/src/main/resources/docs/org.apache.nifi.processors.script.ExecuteScript/index.html b/nar-bundles/execute-script-bundle/execute-script-processors/src/main/resources/docs/org.apache.nifi.processors.script.ExecuteScript/index.html
new file mode 100644
index 0000000..acb47c5
--- /dev/null
+++ b/nar-bundles/execute-script-bundle/execute-script-processors/src/main/resources/docs/org.apache.nifi.processors.script.ExecuteScript/index.html
@@ -0,0 +1,264 @@
+<!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>ExecuteScript</title>
+
+        <link rel="stylesheet" href="../../css/component-usage.css" type="text/css" />
+    </head>
+
+    <body>
+        <!-- Processor Documentation ================================================== -->
+        <h2>Description:</h2>
+        <p>
+            This processor provides the capability to execute scripts in various scripting languages, and passes into the scripts 
+            the input stream and output stream(s) representing an incoming flow file and any created flow files. The processor is designed to be
+            thread safe, so multiple concurrent tasks may execute against a single script. The processor provides a framework which enables 
+            script writers to implement 3 different types of scripts:
+        <ul>
+            ReaderScript - which enables stream-based reading of a FlowFile's content</br>
+            WriterScript - which enables stream-based reading and writing/modifying of a FlowFile's content</br>
+            ConverterScript - which enables stream-based reading a FlowFile's content and stream-based writing to newly created FlowFiles</br>
+        </ul>
+        Presently, the processor supports 3 scripting languages: Ruby, Python, and JavaScript. The processor is built on the 
+        javax.script API which enables ScriptEngine discovery, thread management, and encapsulates much of the low level bridging-code that 
+        enables Java to Script language integration. Thus, it is designed to be easily extended to other scripting languages. </br>
+        The attributes of a FlowFile and properties of the Processor are exposed to the script by either a variable in the base class or 
+        a getter method. A script may declare new Processor Properties and different Relationships via overriding the getPropertyDescriptors 
+        and getRelationships methods, respectively. 
+    </p>
+    The processor provides some boilerplate script to aid in the creation of the three different types of scripts. For example,
+    the processor provides import statements for classes commonly used within a processor.
+    <pre>
+                'org.apache.nifi.components.PropertyDescriptor'
+                'org.apache.nifi.components.Validator'
+                'org.apache.nifi.processor.util.StandardValidators'
+                'org.apache.nifi.processor.Relationship'
+                'org.apache.nifi.logging.ProcessorLog'
+                'org.apache.nifi.scripting.ReaderScript'
+                'org.apache.nifi.scripting.WriterScript'
+                'org.apache.nifi.scripting.ConverterScript'
+    </pre>
+    The processor appends to the script's execution path the parent directory of the specified script file and a sub-directory
+    called 'lib', which may be useful for supporting scripts. </p>
+<p>
+    <strong>Shared Variables</strong>
+</p>
+The following variables are provided as shared variables for the scripts:
+<ul>
+    <li>logger 
+        <ul>
+            <li> The processor's logger </li>
+            <li> Scope is GLOBAL, thus in Ruby the syntax is $logger</li>
+        </ul>
+    </li>
+    <li>properties
+        <ul>
+            <li> A Map of the processor's configuration properties; key and value are strings</li>
+            <li> Scope is INSTANCE, thus in Ruby the syntax is @properties</li>
+        </ul>
+    </li>
+</ul>
+<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. Of particular note: This processor allows scripts to define additional 
+    Processor properties, which will not be initially visible. Once the processor's configuration is validated, script defined properties
+    will become visible, and may affect the validity of the processor.
+</p>
+<ul>
+    <li>
+        <strong>Script File Name</strong>
+        <ul>
+            <li>Script location, can be relative or absolute path.</li>
+            <li>Default value: no default</li>
+            <li>Supports expression language: false</li>
+        </ul>
+    </li>
+    <li>
+        <strong>Script Check Interval</strong>
+        <ul>
+            <li>The time period between checking for updates to a script.</li>
+            <li>Default value: 15 sec</li>
+            <li>Supports expression language: false</li>
+        </ul>
+    </li>
+</ul>
+
+<p>
+    <strong>Relationships:</strong>
+</p>
+<p>
+    The initial 'out of the box' relationships are below. Of particular note is the ability of a script to change the set of
+    relationships. However, any relationships defined by the script will not be visible until the processor's configuration has been 
+    validated. Once done, new relationships will become visible.
+</p>
+<ul>
+    <li>
+        success
+        <ul>
+            <li>Used when a file is successfully processed by a script.</li>
+        </ul>   
+    </li>
+    <li>
+        failure
+        <ul>
+            <li>Used when an error occurs while processing a file with a script.</li>
+        </ul>   
+    </li>
+</ul>
+
+<p>
+    <strong>Example Scripts:</strong>
+</p>
+<ul>
+    JavaScript example - the 'with' statement imports packages defined in the framework and limits the importing to the local scope, 
+    rather than global. The 'Scripting' variable uses the JavaImporter class within JavaScript. Since the 'instance' variable is intended to 
+    be local scope (not global), it must be named 'instance' as it it not passed back to the processor upon script evaluation and must be 
+    fetched. If you make it global, you can name it whatever you'd like...but this is intended to be multi-threaded so do so at your own 
+    risk.</p>
+Presently, there are issues with the JavaScript scripting engine that prevent sub-classing the base classes in the Processor's Java 
+framework. So, what is actually happening is an instance of the ReaderScript is created with a provided callback object. When we are able
+to move to a more competent scripting engine (supposedly in Java 8), the code below will remain the same, but the 'instance' variable 
+will actually be a sub-class of ReaderScript.
+<pre> 
+              with (Scripting) {
+                var instance = new ReaderScript({
+                    route : function(input) {
+                        var str = IOUtils.toString(input);
+                        var expr = instance.getProperty("expr");
+                        filename = instance.attributes.get("filename");
+                        instance.setAttribute("filename", filename + ".modified");
+                        if (str.match(expr)) {
+                            return Script.FAIL_RELATIONSHIP;
+                        } else {
+                            return Script.SUCCESS_RELATIONSHIP;
+                        }
+                    }
+                });
+              }
+</pre>
+Ruby example - the 'OutputStreamHandler' is an interface which is called when creating flow files.
+<pre>
+                java_import 'org.apache.nifi.scripting.OutputStreamHandler'
+                class SimpleConverter < ConverterScript
+                  field_reader :FAIL_RELATIONSHIP, :SUCCESS_RELATIONSHIP, :logger, :attributes
+                  
+                  def convert(input)
+                    in_io = input.to_io
+                    createFlowFile("firstLine", FAIL_RELATIONSHIP, OutputStreamHandler.impl do |method, out|
+                        out_io = out.to_io
+                        out_io << in_io.readline.to_java_bytes
+                        out_io.close
+                        logger.debug("Wrote data to failure...this message logged with logger from super class")
+                      end)
+                      
+                    createFlowFile("otherLines", SUCCESS_RELATIONSHIP, OutputStreamHandler.impl do |method, out|
+                        out_io = out.to_io
+                        in_io.each_line { |line|
+                          out_io << line
+                        }
+                        out_io.close
+                        logger.debug("Wrote data to success...this message logged with logger from super class")
+                      end)
+                    in_io.close
+                  end
+                   
+                end
+                
+                $logger.debug("Creating SimpleConverter...this message logged with logger from shared variables")
+                SimpleConverter.new
+</pre>
+Python example - The difficulty with Python is that it does not return objects upon script evaluation, so the instance of the Script 
+class must be fetched by name. Thus, you must define a variable called 'instance'.
+<pre>
+                import re
+                
+                class RoutingReader(ReaderScript):
+                    A = Relationship.Builder().name("a").description("some good stuff").build()
+                    B = Relationship.Builder().name("b").description("some other stuff").build()
+                    C = Relationship.Builder().name("c").description("some bad stuff").build()
+                    
+                    def getRelationships(self):
+                        return [self.A,self.B,self.C]
+                  
+                    def getExceptionRoute(self):
+                        return self.C
+                  
+                    def route( self, input ):
+                        logger.info("Executing route")
+                        for line in FileUtil.wrap(input):
+                            if re.match("^bad", line, re.IGNORECASE):
+                                return self.B
+                            if re.match("^sed", line):
+                                raise RuntimeError("That's no good!")
+                
+                        return self.A
+                logger.debug("Constructing instance")
+                instance = RoutingReader()
+            
+</pre>
+</ul>
+<p>
+    <strong>Script API:</strong>
+</p>
+<ul>
+    <li>getAttribute(String) : String</li>
+    <li>getAttributes() : Map(String,String)</li>
+    <li>getExceptionRoute() : Relationship</li>
+    <li>getFileName() : String</li>
+    <li>getFlowFileEntryDate() : Calendar</li>
+    <li>getFlowFileSize() : long</li>
+    <li>getProperties() : Map(String, String)</li>
+    <li>getProperty(String) : String</li>
+    <li>getPropertyDescriptors() : List(PropertyDescriptor)</li>
+    <li>getRelationships() : Collection(Relationship)</li>
+    <li>getRoute() : Relationship</li>
+    <li>setRoute(Relationship)</li>
+    <li>setAttribute(String, String)</li>
+    <li>validate() : Collection(String)</li>
+</ul>
+<p>
+    <strong>ReaderScript API:</strong>
+</p>
+<ul>
+    <li>route(InputStream) : Relationship</li>
+</ul>
+<p>
+    <strong>WriterScript API:</strong>
+</p>
+<ul>
+    <li>process(InputStream, OutputStream)</li>
+</ul>
+<p>
+    <strong>ConverterScript API:</strong>
+</p>
+<ul>
+    <li>convert(InputStream)</li>
+    <li>createFlowFile(String, Relationship, OutputStreamHandler)</li>
+</ul>
+<p>
+    <strong>OutputStreamHandler API:</strong>
+</p>
+<ul>
+    <li>write(OutputStream)</li>
+</ul>
+</body>
+</html>

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/nar-bundles/execute-script-bundle/execute-script-processors/src/test/java/org/apache/nifi/processors/script/TestExecuteScript.java
----------------------------------------------------------------------
diff --git a/nar-bundles/execute-script-bundle/execute-script-processors/src/test/java/org/apache/nifi/processors/script/TestExecuteScript.java b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/java/org/apache/nifi/processors/script/TestExecuteScript.java
new file mode 100644
index 0000000..3a34769
--- /dev/null
+++ b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/java/org/apache/nifi/processors/script/TestExecuteScript.java
@@ -0,0 +1,939 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.processors.script;
+
+import org.apache.nifi.processors.script.ExecuteScript;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.util.MockFlowFile;
+import org.apache.nifi.util.TestRunner;
+import org.apache.nifi.util.TestRunners;
+
+import org.apache.commons.io.FileUtils;
+import org.junit.Before;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * @author unattributed
+ *
+ */
+public class TestExecuteScript {
+
+    static Logger LOG;
+
+    static {
+        System.setProperty("org.slf4j.simpleLogger.defaultLogLevel", "info");
+        System.setProperty("org.slf4j.simpleLogger.showDateTime", "true");
+        System.setProperty("org.slf4j.simpleLogger.log.nifi.processors.script.ExecuteScript", "trace");
+        System.setProperty("org.slf4j.simpleLogger.log.nifi.processors.script.TestExecuteScript", "debug");
+        System.setProperty("org.slf4j.simpleLogger.log.nifi.processors.AbstractProcessor", "debug");
+        LOG = LoggerFactory.getLogger(TestExecuteScript.class);
+    }
+
+    private TestRunner controller;
+
+    private final String multiline = "Lorem ipsum dolor sit amet,\n"
+            + "consectetur adipisicing elit,\n"
+            + "sed do eiusmod tempor incididunt ut labore et dolore magna aliqua.\n"
+            + "Ut enim ad minim veniam,\n"
+            + "quis nostrud exercitation ullamco laboris nisi ut aliquip ex ea commodo consequat.\n"
+            + "Duis aute irure dolor in reprehenderit in voluptate velit esse cillum dolore eu fugiat nulla pariatur.\n"
+            + "Excepteur sint occaecat cupidatat non proident,\n"
+            + "sunt in culpa qui officia deserunt mollit anim id est laborum.";
+
+    /**
+     * Create a mock SingleProcessorController using our processor and pass data
+     * to it via byte array. Returns the Sink that provides access to any files
+     * that pass out of the processor
+     */
+    @Before
+    public void setupEach() throws IOException {
+        controller = TestRunners.newTestRunner(ExecuteScript.class);
+        controller.setValidateExpressionUsage(false);
+
+        // copy all scripts to target directory and run from there. some python
+        // scripts create .class files that end up in src/test/resources.
+        FileUtils.copyDirectory(new File("src/test/resources"), new File("target/test-scripts"));
+    }
+
+    // Fail if the specified relationship does not contain exactly one file
+    // with the expected value
+    private void assertRelationshipContents(String expected, String relationship) {
+        controller.assertTransferCount(relationship, 1);
+        MockFlowFile ff = controller.getFlowFilesForRelationship(relationship).get(0);
+        ff.assertContentEquals(expected);
+    }
+
+    // Fail if the specified relationship does not contain specified number of files
+    // with the expected value
+    private void assertRelationshipContents(String expected, String relationship, int count) {
+        controller.assertTransferCount(relationship, count);
+        MockFlowFile ff = controller.getFlowFilesForRelationship(relationship).get(count - 1);
+        ff.assertContentEquals(expected);
+    }
+
+    // ////////////////////////////////////
+    // General tests
+    @Test(expected = IllegalArgumentException.class)
+    public void failOnBadName() {
+        LOG.info("Supplying bad script file names");
+
+        // None of these should result in actually setting the property, because they're non-existent / bad files
+        controller.setProperty(ExecuteScript.SCRIPT_FILE_NAME, "not/really.rb");
+        controller.assertNotValid();
+        controller.setProperty(ExecuteScript.SCRIPT_FILE_NAME, "fakey/fake.js");
+        controller.assertNotValid();
+        controller.setProperty(ExecuteScript.SCRIPT_FILE_NAME, "pom.xml");
+        controller.assertNotValid();
+    }
+
+    // ////////////////////////////////////
+    // Ruby script tests
+    @Test
+    public void testSimpleReadR() {
+        LOG.info("Ruby script: fail file based on reading contents");
+
+        controller.enqueue("This stuff is fine".getBytes());
+        controller.enqueue(multiline.getBytes());
+
+        controller.setProperty(ExecuteScript.SCRIPT_FILE_NAME, "target/test-scripts/readTest.rb");
+        controller.setThreadCount(2);
+        controller.run(2);
+
+        assertRelationshipContents(multiline, "failure");
+        assertRelationshipContents("This stuff is fine", "success");
+
+        controller.getFlowFilesForRelationship("success").get(0).assertAttributeEquals("filename", "NewFileNameFromReadTest");
+    }
+
+    @Test
+    public void testParamReadR() {
+        LOG.info("Ruby script: Failing file based on reading contents");
+
+        Map<String, String> attrs1 = new HashMap<>();
+        attrs1.put("filename", "StuffIsFine.txt");
+        Map<String, String> attrs2 = new HashMap<>();
+        attrs2.put("filename", "multiline.txt");
+        controller.enqueue("This stuff is fine".getBytes(), attrs1);
+        controller.enqueue(multiline.getBytes(), attrs2);
+
+        controller.setProperty(ExecuteScript.SCRIPT_FILE_NAME, "target/test-scripts/readWithParams.rb");
+        controller.setProperty("expr", "rehenderit");
+        controller.run(2);
+
+        assertRelationshipContents(multiline, "failure");
+        assertRelationshipContents("This stuff is fine", "success");
+    }
+
+    @Test
+    public void testWriteLastLineR() {
+        LOG.info("Running Ruby script to output last line of file");
+
+        controller.enqueue(multiline.getBytes());
+
+        controller.setProperty(ExecuteScript.SCRIPT_FILE_NAME, "target/test-scripts/writeTest.rb");
+        controller.run();
+
+        List<MockFlowFile> files = controller.getFlowFilesForRelationship("success");
+
+        assertEquals("Process did not generate an output file", 1, files.size());
+
+        byte[] blob = files.get(0).toByteArray();
+        String[] lines = new String(blob).split("\n");
+
+        assertEquals("File had more than one line", 1, lines.length);
+        assertEquals("sunt in culpa qui officia deserunt mollit anim id est laborum.", lines[0]);
+    }
+
+    @Test
+    public void testWriteOptionalParametersR() {
+        LOG.info("Ruby script that uses optional parameters");
+
+        controller.enqueue(multiline.getBytes());
+
+        controller.setProperty(ExecuteScript.SCRIPT_FILE_NAME, "target/test-scripts/paramTest.rb");
+        controller.setProperty("repeat", "3");
+        controller.run();
+
+        List<MockFlowFile> files = controller.getFlowFilesForRelationship("success");
+
+        assertEquals("Process did not generate an output file", 1, files.size());
+
+        byte[] blob = files.get(0).toByteArray();
+        String[] lines = new String(blob).split("\n");
+
+        assertEquals("File did not have 3 lines", 3, lines.length);
+        assertEquals("sunt in culpa qui officia deserunt mollit anim id est laborum.", lines[0]);
+    }
+
+    @Test
+    public void testSetupOptionalValidationR() {
+        LOG.info("Ruby script creating validators for optional properties");
+
+        controller.setProperty(ExecuteScript.SCRIPT_FILE_NAME, "target/test-scripts/optionalValidators.rb");
+        controller.assertNotValid();
+        controller.setProperty("int", "abc");
+        controller.assertNotValid();
+        controller.setProperty("url", "not@valid");
+        controller.assertNotValid();
+        controller.setProperty("nonEmpty", "");
+        controller.assertNotValid();
+
+        controller.setProperty("int", "123");
+        controller.setProperty("url", "http://localhost");
+        controller.setProperty("nonEmpty", "abc123");
+        controller.assertValid();
+    }
+
+    @Test
+    public void testTwoScriptsSameThreadSameClassName() {
+        LOG.info("Test 2 different scripts with the same ruby class name");
+
+        Map<String, String> attrs1 = new HashMap<>();
+        attrs1.put("filename", "StuffIsFine.txt");
+        Map<String, String> attrs2 = new HashMap<>();
+        attrs2.put("filename", "multiline.txt");
+
+        controller.enqueue("This stuff is fine".getBytes(), attrs1);
+        controller.enqueue(multiline.getBytes(), attrs2);
+
+        controller.setProperty(ExecuteScript.SCRIPT_FILE_NAME, "target/test-scripts/readWithParams.rb");
+        controller.setProperty("expr", "rehenderit");
+        controller.run(2);
+
+        assertRelationshipContents(multiline, "failure");
+        assertRelationshipContents("This stuff is fine", "success");
+
+        controller.setProperty(ExecuteScript.SCRIPT_FILE_NAME, "target/test-scripts/optionalValidators.rb");
+        controller.assertNotValid();
+        controller.setProperty("int", "abc");
+        controller.assertNotValid();
+        controller.setProperty("url", "not@valid");
+        controller.assertNotValid();
+        controller.setProperty("nonEmpty", "");
+        controller.assertNotValid();
+
+        controller.setProperty("int", "123");
+        controller.setProperty("url", "http://localhost");
+        controller.setProperty("nonEmpty", "abc123");
+        controller.assertValid();
+    }
+
+    @Test
+    public void testUpdateScriptR() throws Exception {
+        LOG.info("Test one script with updated class");
+
+        File testFile = File.createTempFile("script", ".rb");
+        File original = new File("target/test-scripts/readWithParams.rb");
+        FileUtils.copyFile(original, testFile);
+        controller.setProperty(ExecuteScript.SCRIPT_FILE_NAME, testFile.getPath());
+        controller.assertValid();
+        original = new File("target/test-scripts/optionalValidators.rb");
+        FileUtils.copyFile(original, testFile);
+        controller.setProperty(ExecuteScript.SCRIPT_CHECK_INTERVAL, "5 secs");
+        Thread.sleep(6000);
+
+        controller.assertNotValid();
+        controller.setProperty("int", "abc");
+        controller.assertNotValid();
+        controller.setProperty("url", "not@valid");
+        controller.assertNotValid();
+        controller.setProperty("nonEmpty", "");
+        controller.assertNotValid();
+
+        controller.setProperty("int", "123");
+        controller.setProperty("url", "http://localhost");
+        controller.setProperty("nonEmpty", "abc123");
+        controller.assertValid();
+        FileUtils.deleteQuietly(testFile);
+    }
+
+    @Test
+    public void testMultiThreadExecR() {
+        LOG.info("Ruby script 20 threads: Failing file based on reading contents");
+
+        Map<String, String> attrs1 = new HashMap<>();
+        attrs1.put("filename", "StuffIsFine.txt");
+        Map<String, String> attrs2 = new HashMap<>();
+        attrs2.put("filename", "multiline.txt");
+        controller.setThreadCount(20);
+        for (int i = 0; i < 10; i++) {
+            controller.enqueue("This stuff is fine".getBytes(), attrs1);
+            controller.enqueue(multiline.getBytes(), attrs2);
+        }
+
+        controller.setProperty(ExecuteScript.SCRIPT_FILE_NAME, "target/test-scripts/readWithParams.rb");
+        controller.setProperty("expr", "rehenderit");
+        controller.run(20);
+
+        controller.assertTransferCount("failure", 10);
+        controller.assertTransferCount("success", 10);
+        for (int i = 0; i < 10; i++) {
+            MockFlowFile ff = controller.getFlowFilesForRelationship("failure").get(i);
+            ff.assertContentEquals(multiline);
+            assertTrue(ff.getAttribute("filename").endsWith("modified"));
+            ff = controller.getFlowFilesForRelationship("success").get(i);
+            ff.assertContentEquals("This stuff is fine");
+            assertTrue(ff.getAttribute("filename").endsWith("modified"));
+        }
+
+    }
+
+    @Test
+    public void testManualValidationR() {
+        LOG.info("Ruby script defining manual validator");
+
+        controller.setProperty(ExecuteScript.SCRIPT_FILE_NAME, "target/test-scripts/alwaysFail.rb");
+        controller.assertNotValid();
+    }
+
+    @Test
+    public void testGetRelationshipsR() {
+        LOG.info("Ruby script: getRelationships");
+        controller.setProperty(ExecuteScript.SCRIPT_FILE_NAME, "target/test-scripts/routeTest.rb");
+        // at this point, the script has not been instantiated so the processor simply returns an empty set
+        Set<Relationship> rels = controller.getProcessor().getRelationships();
+        assertEquals(0, rels.size());
+        // this will instantiate the script
+        controller.assertValid();
+        // this will call the script
+        rels = controller.getProcessor().getRelationships();
+        assertEquals(3, rels.size());
+    }
+
+    @Test
+    public void testGetExceptionRouteR() {
+        LOG.info("Ruby script defining route taken in event of exception");
+
+        controller.enqueue("This stuff is fine".getBytes());
+        controller.enqueue("Bad things go to 'b'.".getBytes());
+        controller.enqueue(multiline.getBytes());
+
+        controller.setProperty(ExecuteScript.SCRIPT_FILE_NAME, "target/test-scripts/routeTest.rb");
+
+        controller.run(3);
+
+        assertRelationshipContents("This stuff is fine", "a");
+        assertRelationshipContents("Bad things go to 'b'.", "b");
+        assertRelationshipContents(multiline, "c");
+
+    }
+
+    @Test
+    public void testSimpleConverterR() {
+        LOG.info("Running Ruby converter script");
+
+        for (int i = 0; i < 20; i++) {
+            controller.enqueue(multiline.getBytes());
+        }
+
+        controller.setThreadCount(20);
+        controller.setProperty(ExecuteScript.SCRIPT_FILE_NAME, "target/test-scripts/simpleConverter.rb");
+        controller.run(20);
+
+        List<MockFlowFile> successFiles = controller.getFlowFilesForRelationship("success");
+        List<MockFlowFile> failFiles = controller.getFlowFilesForRelationship("failure");
+
+        assertEquals("Process did not generate 20 SUCCESS files", 20, successFiles.size());
+        assertEquals("Process did not generate 20 FAILURE files", 20, failFiles.size());
+
+        MockFlowFile sFile = successFiles.get(19);
+        MockFlowFile fFile = failFiles.get(19);
+
+        byte[] blob = fFile.toByteArray();
+        String[] lines = new String(blob).split("\n");
+
+        assertEquals("File had more than one line", 1, lines.length);
+        assertEquals("Lorem ipsum dolor sit amet,", lines[0]);
+
+        blob = sFile.toByteArray();
+        lines = new String(blob).split("\n");
+
+        assertEquals("SUCCESS had wrong number of lines", 7, lines.length);
+        assertEquals("consectetur adipisicing elit,", lines[0]);
+    }
+
+    @Test
+    public void testLoadLocalR() {
+        LOG.info("Ruby: load another script file");
+
+        controller.enqueue("This stuff is fine".getBytes());
+        controller.enqueue(multiline.getBytes());
+
+        controller.setProperty(ExecuteScript.SCRIPT_FILE_NAME, "target/test-scripts/loadLocal.rb");
+        controller.run(2);
+
+        assertRelationshipContents(multiline, "failure");
+        assertRelationshipContents("This stuff is fine", "success");
+    }
+
+    @Test
+    public void testFlowFileR() {
+        LOG.info("Ruby: get FlowFile properties");
+
+        controller.enqueue(multiline.getBytes());
+        HashMap<String, String> meta = new HashMap<String, String>();
+        meta.put("evict", "yup");
+        controller.enqueue("This would be plenty long but it's also evicted.".getBytes(), meta);
+        controller.enqueue("This is too short".getBytes());
+
+        controller.setProperty(ExecuteScript.SCRIPT_FILE_NAME, "target/test-scripts/ffTest.rb");
+        controller.run(3);
+
+        assertRelationshipContents(multiline, "success");
+        assertRelationshipContents("This is too short", "failure");
+        assertRelationshipContents("This would be plenty long but it's also evicted.", "evict");
+    }
+
+    // //////////////////////////////////// // JS tests
+    @Test
+    public void testSimpleReadJS() {
+        LOG.info("Javascript: fail file based on reading contents");
+
+        controller.enqueue("This stuff is fine".getBytes());
+        controller.enqueue(multiline.getBytes());
+
+        controller.setProperty(ExecuteScript.SCRIPT_FILE_NAME, "target/test-scripts/readTest.js");
+        controller.run(2);
+
+        assertRelationshipContents(multiline, "failure");
+        assertRelationshipContents("This stuff is fine", "success");
+    }
+
+    @Test
+    public void testParamReadJS() {
+        LOG.info("Javascript: read contents and fail based on parameter");
+
+        controller.enqueue("This stuff is fine".getBytes());
+        controller.enqueue(multiline.getBytes());
+
+        controller.setProperty(ExecuteScript.SCRIPT_FILE_NAME, "target/test-scripts/readWithParams.js");
+        controller.setProperty("expr", "sed do");
+        controller.run(2);
+
+        assertRelationshipContents(multiline, "failure");
+        assertRelationshipContents("This stuff is fine", "success");
+    }
+
+    @Test
+    public void testWriteLastLineJS() {
+        LOG.info("Running Javascript to output last line of file");
+
+        controller.enqueue(multiline.getBytes());
+
+        controller.setProperty(ExecuteScript.SCRIPT_FILE_NAME, "target/test-scripts/writeTest.js");
+        controller.run();
+
+        List<MockFlowFile> sunkFiles = controller.getFlowFilesForRelationship("success");
+
+        assertEquals("Process did not generate an output file", 1, sunkFiles.size());
+
+        MockFlowFile sunkFile = sunkFiles.iterator().next();
+        byte[] blob = sunkFile.toByteArray();
+        String[] lines = new String(blob).split("\n");
+
+        assertEquals("File had more than one line", 1, lines.length);
+        assertEquals("sunt in culpa qui officia deserunt mollit anim id est laborum.", lines[0]);
+    }
+
+    @Test
+    public void testWriteOptionalParametersJS() {
+        LOG.info("Javascript processCallback that uses optional parameters");
+
+        controller.enqueue(multiline.getBytes());
+
+        controller.setProperty(ExecuteScript.SCRIPT_FILE_NAME, "target/test-scripts/paramTest.js");
+        controller.setProperty("repeat", "3");
+        controller.run();
+
+        List<MockFlowFile> sunkFiles = controller.getFlowFilesForRelationship("success");
+
+        assertEquals("Process did not generate an output file", 1, sunkFiles.size());
+
+        MockFlowFile sunkFile = sunkFiles.iterator().next();
+        byte[] blob = sunkFile.toByteArray();
+        String[] lines = new String(blob).split("\n");
+
+        assertEquals("File did not have 3 lines", 3, lines.length);
+        assertEquals("sunt in culpa qui officia deserunt mollit anim id est laborum.", lines[0]);
+    }
+
+    @Test
+    public void testSetupOptionalValidationJS() {
+        LOG.info("Javascript creating validators for optional properties");
+
+        controller.setProperty(ExecuteScript.SCRIPT_FILE_NAME, "target/test-scripts/optionalValidators.js");
+        controller.setProperty("int", "abc");
+        controller.setProperty("url", "not@valid");
+        controller.setProperty("nonEmpty", "");
+        assertEquals(2, controller.getProcessor().getPropertyDescriptors().size());
+        controller.assertNotValid(); // due to invalid values above
+        assertEquals(5, controller.getProcessor().getPropertyDescriptors().size());
+
+        controller.setProperty("int", "123");
+        controller.setProperty("url", "http://localhost");
+        controller.setProperty("nonEmpty", "abc123");
+        assertEquals(5, controller.getProcessor().getPropertyDescriptors().size());
+        controller.assertValid();
+    }
+
+    @Test
+    public void testManualValidationJS() {
+        LOG.info("Javascript defining manual validator");
+
+        controller.setProperty(ExecuteScript.SCRIPT_FILE_NAME, "target/test-scripts/alwaysFail.js");
+        controller.assertNotValid();
+    }
+
+    @Test
+    public void testGetExceptionRouteJS() {
+        LOG.info("Javascript defining route taken in event of exception");
+
+        controller.enqueue("This stuff is fine".getBytes());
+        controller.enqueue("Bad things go to 'b'.".getBytes());
+        controller.enqueue(multiline.getBytes());
+
+        controller.setProperty(ExecuteScript.SCRIPT_FILE_NAME, "target/test-scripts/routeTest.js");
+
+        controller.run(3);
+
+        assertRelationshipContents("This stuff is fine", "a");
+        assertRelationshipContents("Bad things go to 'b'.", "b");
+        assertRelationshipContents(multiline, "c");
+
+    }
+
+    @Test
+    public void testSimpleConverterJS() {
+        LOG.info("Running Javascript converter script");
+
+        for (int i = 0; i < 20; i++) {
+            controller.enqueue(multiline.getBytes());
+        }
+
+        controller.setProperty(ExecuteScript.SCRIPT_FILE_NAME, "target/test-scripts/simpleConverter.js");
+        controller.run(20);
+
+        List<MockFlowFile> successFiles = controller.getFlowFilesForRelationship("success");
+        List<MockFlowFile> failFiles = controller.getFlowFilesForRelationship("failure");
+
+        assertEquals("Process did not generate 20 SUCCESS files", 20, successFiles.size());
+        assertEquals("Process did not generate 20 FAILURE file", 20, failFiles.size());
+
+        MockFlowFile sFile = successFiles.get(19);
+        MockFlowFile fFile = failFiles.get(0);
+
+        byte[] blob = sFile.toByteArray();
+        String[] lines = new String(blob).split("\n");
+
+        assertEquals("SUCCESS had wrong number of lines", 7, lines.length);
+        assertTrue(lines[0].startsWith("consectetur adipisicing elit,"));
+
+        blob = fFile.toByteArray();
+        lines = new String(blob).split("\n");
+
+        assertEquals("File had more than one line", 1, lines.length);
+        assertTrue(lines[0].startsWith("Lorem ipsum dolor sit amet,"));
+    }
+
+    @Test
+    public void testLoadLocalJS() {
+        LOG.info("Javascript: load another script file");
+
+        controller.enqueue("This stuff is fine".getBytes());
+        controller.enqueue(multiline.getBytes());
+
+        controller.setProperty(ExecuteScript.SCRIPT_FILE_NAME, "target/test-scripts/loadLocal.js");
+        controller.run(2);
+
+        assertRelationshipContents(multiline, "failure");
+        assertRelationshipContents("This stuff is fine", "success");
+    }
+
+    @Test
+    public void testXMLJS() {
+        LOG.info("Javascript: native XML parser");
+
+        controller.enqueue("<a><b foo='bar'>Bad</b><b good='true'>Good</b><b good='false'>Bad</b></a>".getBytes());
+        controller.enqueue("<a><b>Hello</b><b>world</b></a>".getBytes());
+
+        controller.setProperty(ExecuteScript.SCRIPT_FILE_NAME, "target/test-scripts/parseXml.js");
+        controller.run(2);
+
+        assertRelationshipContents("Good", "success");
+        assertRelationshipContents("<a><b>Hello</b><b>world</b></a>", "failure");
+    }
+
+    @Test
+    public void testFlowFileJS() {
+        LOG.info("JavaScript: get FlowFile properties");
+
+        controller.enqueue("This is too short".getBytes());
+        controller.enqueue(multiline.getBytes());
+
+        controller.setProperty(ExecuteScript.SCRIPT_FILE_NAME, "target/test-scripts/ffTest.js");
+        controller.run(2);
+
+        assertRelationshipContents(multiline, "success");
+        assertRelationshipContents("This is too short", "failure");
+    }
+
+    @Test
+    public void testMultiThreadExecJS() {
+        LOG.info("JavaScript script 20 threads: Failing file based on reading contents");
+
+        Map<String, String> attrs1 = new HashMap<>();
+        attrs1.put("filename", "StuffIsFine.txt");
+        Map<String, String> attrs2 = new HashMap<>();
+        attrs2.put("filename", "multiline.txt");
+        controller.setThreadCount(20);
+        for (int i = 0; i < 10; i++) {
+            controller.enqueue("This stuff is fine".getBytes(), attrs1);
+            controller.enqueue(multiline.getBytes(), attrs2);
+        }
+
+        controller.setProperty(ExecuteScript.SCRIPT_FILE_NAME, "target/test-scripts/readWithParams.js");
+        controller.setProperty("expr", "rehenderit");
+        controller.run(20);
+
+        controller.assertTransferCount("failure", 10);
+        controller.assertTransferCount("success", 10);
+        for (int i = 0; i < 10; i++) {
+            MockFlowFile ff = controller.getFlowFilesForRelationship("failure").get(i);
+            ff.assertContentEquals(multiline);
+            assertTrue(ff.getAttribute("filename").endsWith("modified"));
+            ff = controller.getFlowFilesForRelationship("success").get(i);
+            ff.assertContentEquals("This stuff is fine");
+            assertTrue(ff.getAttribute("filename").endsWith("modified"));
+        }
+    }
+
+    @Test
+    public void testUpdateScriptJS() throws Exception {
+        LOG.info("Test one script with updated class");
+
+        File testFile = File.createTempFile("script", ".js");
+        File original = new File("target/test-scripts/readWithParams.js");
+        FileUtils.copyFile(original, testFile);
+        controller.setProperty(ExecuteScript.SCRIPT_FILE_NAME, testFile.getPath());
+        controller.assertValid();
+        original = new File("target/test-scripts/optionalValidators.js");
+        FileUtils.copyFile(original, testFile);
+        controller.setProperty(ExecuteScript.SCRIPT_CHECK_INTERVAL, "5 secs");
+        Thread.sleep(6000);
+
+        controller.assertNotValid();
+        controller.setProperty("int", "abc");
+        controller.assertNotValid();
+        controller.setProperty("url", "not@valid");
+        controller.assertNotValid();
+        controller.setProperty("nonEmpty", "");
+        controller.assertNotValid();
+
+        controller.setProperty("int", "123");
+        controller.setProperty("url", "http://localhost");
+        controller.setProperty("nonEmpty", "abc123");
+        controller.assertValid();
+        FileUtils.deleteQuietly(testFile);
+    }
+
+    // ////////////////////////////////// // Python script tests
+    @Test
+    public void testSimpleReadP() {
+        LOG.info("Python script: fail file based on reading contents");
+
+        for (int i = 0; i < 20; i++) {
+            Map<String, String> attr1 = new HashMap<>();
+            attr1.put("filename", "FineStuff");
+            attr1.put("counter", Integer.toString(i));
+            Map<String, String> attr2 = new HashMap<>();
+            attr2.put("filename", "MultiLine");
+            attr2.put("counter", Integer.toString(i));
+            controller.enqueue("This stuff is fine".getBytes(), attr1);
+            controller.enqueue(multiline.getBytes(), attr2);
+        }
+
+        controller.setThreadCount(40);
+        controller.setProperty(ExecuteScript.SCRIPT_FILE_NAME, "target/test-scripts/readTest.py");
+        controller.run(40);
+
+        assertRelationshipContents(multiline, "failure", 20);
+        assertRelationshipContents("This stuff is fine", "success", 20);
+
+        List<MockFlowFile> fails = controller.getFlowFilesForRelationship("failure");
+        List<MockFlowFile> successes = controller.getFlowFilesForRelationship("success");
+        for (int i = 0; i < 20; i++) {
+            assertTrue(fails.get(i).getAttribute("filename").matches("^.*\\d+$"));
+            assertTrue(successes.get(i).getAttribute("filename").matches("^.*\\d+$"));
+        }
+    }
+
+    @Test
+    public void testParamReadP() {
+        LOG.info("Python script: read contents and fail based on parameter");
+
+        controller.enqueue("This stuff is fine".getBytes());
+        controller.enqueue(multiline.getBytes());
+
+        controller.setProperty(ExecuteScript.SCRIPT_FILE_NAME, "target/test-scripts/readWithParams.py");
+        controller.setProperty("expr", "sed do");
+        controller.run(2);
+
+        assertRelationshipContents(multiline, "failure");
+        assertRelationshipContents("This stuff is fine", "success");
+    }
+
+    @Test
+    public void testWriteLastLineP() {
+        LOG.info("Running Python script to output last line of file");
+
+        controller.enqueue(multiline.getBytes());
+
+        controller.setProperty(ExecuteScript.SCRIPT_FILE_NAME, "target/test-scripts/writeTest.py");
+        controller.run();
+
+        List<MockFlowFile> sunkFiles = controller.getFlowFilesForRelationship("success");
+
+        assertEquals("Process did not generate an output file", 1, sunkFiles.size());
+
+        MockFlowFile sunkFile = sunkFiles.iterator().next();
+        byte[] blob = sunkFile.toByteArray();
+        String[] lines = new String(blob).split("\n");
+
+        assertEquals("File had more than one line", 1, lines.length);
+        assertEquals("sunt in culpa qui officia deserunt mollit anim id est laborum.", lines[0]);
+    }
+
+    @Test
+    public void testWriteOptionalParametersP() {
+        LOG.info("Python script processCallback that uses optional parameters");
+
+        controller.enqueue(multiline.getBytes());
+
+        controller.setProperty(ExecuteScript.SCRIPT_FILE_NAME, "target/test-scripts/paramTest.py");
+        controller.setProperty("repeat", "3");
+        controller.run();
+
+        List<MockFlowFile> sunkFiles = controller.getFlowFilesForRelationship("success");
+
+        assertEquals("Process did not generate an output file", 1, sunkFiles.size());
+
+        MockFlowFile sunkFile = sunkFiles.iterator().next();
+        byte[] blob = sunkFile.toByteArray();
+        String[] lines = new String(blob).split("\n");
+
+        assertEquals("File did not have 3 lines", 3, lines.length);
+        assertTrue(lines[2].startsWith("sunt in culpa qui officia deserunt mollit anim id est laborum."));
+    }
+
+    @Test
+    public void testManualValidationP() {
+        LOG.info("Python defining manual validator");
+
+        controller.setProperty(ExecuteScript.SCRIPT_FILE_NAME, "target/test-scripts/alwaysFail.py");
+        controller.assertNotValid();
+    }
+
+    @Test
+    public void testSetupOptionalValidationP() {
+        LOG.info("Python script creating validators for optional properties");
+
+        controller.setProperty(ExecuteScript.SCRIPT_FILE_NAME, "target/test-scripts/optionalValidators.py");
+        controller.setProperty("int", "abc");
+        controller.setProperty("url", "not@valid");
+        controller.setProperty("nonEmpty", "");
+        assertEquals(2, controller.getProcessor().getPropertyDescriptors().size());
+        controller.assertNotValid();
+
+        controller.setProperty("int", "123");
+        controller.setProperty("url", "http://localhost");
+        controller.setProperty("nonEmpty", "abc123");
+        assertEquals(5, controller.getProcessor().getPropertyDescriptors().size());
+        controller.assertValid();
+    }
+
+    @Test
+    public void testGetExceptionRouteP() {
+        LOG.info("Python script defining route taken in event of exception");
+
+        controller.enqueue("This stuff is fine".getBytes());
+        controller.enqueue("Bad things go to 'b'.".getBytes());
+        controller.enqueue(multiline.getBytes());
+
+        controller.setProperty(ExecuteScript.SCRIPT_FILE_NAME, "target/test-scripts/routeTest.py");
+
+        // Don't put the error in the logs
+        // TestableAppender ta = new TestableAppender();
+        // ta.attach(Logger.getLogger(ExecuteScript.class));
+        controller.run(3);
+        // ta.detach();
+
+        assertRelationshipContents("This stuff is fine", "a");
+        assertRelationshipContents("Bad things go to 'b'.", "b");
+        assertRelationshipContents(multiline, "c");
+
+        // ta.assertFound("threw exception");
+    }
+
+    @Test
+    public void testLoadLocalP() throws Exception {
+
+        final Thread t = new Thread(new Runnable() {
+            @Override
+            public void run() {
+                try {
+                    testGetExceptionRouteP();
+                    setupEach();
+                } catch (Exception e) {
+
+                }
+            }
+        });
+
+        t.start();
+        t.join();
+
+        LOG.info("Python: load another script file");
+
+        controller.enqueue("This stuff is fine".getBytes());
+        controller.enqueue(multiline.getBytes());
+
+        controller.setProperty(ExecuteScript.SCRIPT_FILE_NAME, "target/test-scripts/loadLocal.py");
+        controller.run(2);
+
+        assertRelationshipContents(multiline, "failure");
+        assertRelationshipContents("This stuff is fine", "success");
+    }
+
+    @Test
+    public void testSimpleConverterP() {
+        LOG.info("Running Python converter script");
+
+        for (int i = 0; i < 20; i++) {
+            controller.enqueue(multiline.getBytes());
+        }
+
+        controller.setThreadCount(20);
+        controller.setProperty(ExecuteScript.SCRIPT_FILE_NAME, "target/test-scripts/simpleConverter.py");
+        controller.run(20);
+
+        List<MockFlowFile> successFiles = controller.getFlowFilesForRelationship("success");
+        List<MockFlowFile> failFiles = controller.getFlowFilesForRelationship("failure");
+
+        assertEquals("Process did not generate 20 SUCCESS files", 20, successFiles.size());
+        assertEquals("Process did not generate 20 FAILURE files", 20, failFiles.size());
+
+        MockFlowFile sFile = successFiles.iterator().next();
+        MockFlowFile fFile = failFiles.iterator().next();
+
+        byte[] blob = sFile.toByteArray();
+        String[] lines = new String(blob).split("\n");
+
+        assertEquals("SUCCESS had wrong number of lines", 7, lines.length);
+        assertTrue(lines[0].startsWith("consectetur adipisicing elit,"));
+
+        blob = fFile.toByteArray();
+        lines = new String(blob).split("\n");
+
+        assertEquals("File had more than one line", 1, lines.length);
+        assertTrue(lines[0].startsWith("Lorem ipsum dolor sit amet,"));
+    }
+
+    @Test
+    public void testFlowFileP() {
+        LOG.info("Python: get FlowFile properties");
+
+        controller.enqueue("This is too short".getBytes());
+        controller.enqueue(multiline.getBytes());
+
+        controller.setProperty(ExecuteScript.SCRIPT_FILE_NAME, "target/test-scripts/ffTest.py");
+        controller.run(2);
+
+        assertRelationshipContents(multiline, "success");
+        assertRelationshipContents("This is too short", "failure");
+    }
+
+    @Test
+    public void testMultiThreadExecP() {
+        LOG.info("Pthon script 20 threads: Failing file based on reading contents");
+
+        Map<String, String> attrs1 = new HashMap<>();
+        attrs1.put("filename", "StuffIsFine.txt");
+        Map<String, String> attrs2 = new HashMap<>();
+        attrs2.put("filename", "multiline.txt");
+        for (int i = 0; i < 10; i++) {
+            controller.enqueue("This stuff is fine".getBytes(), attrs1);
+            controller.enqueue(multiline.getBytes(), attrs2);
+        }
+
+        controller.setThreadCount(20);
+        controller.setProperty(ExecuteScript.SCRIPT_FILE_NAME, "target/test-scripts/readWithParams.py");
+        controller.setProperty("expr", "sed do");
+        controller.run(20);
+
+        controller.assertTransferCount("failure", 10);
+        controller.assertTransferCount("success", 10);
+        for (int i = 0; i < 10; i++) {
+            MockFlowFile ff = controller.getFlowFilesForRelationship("failure").get(i);
+            ff.assertContentEquals(multiline);
+            assertTrue(ff.getAttribute("filename").endsWith("modified"));
+            ff = controller.getFlowFilesForRelationship("success").get(i);
+            ff.assertContentEquals("This stuff is fine");
+            assertTrue(ff.getAttribute("filename").endsWith("modified"));
+        }
+    }
+
+    @Test
+    public void testUpdateScriptP() throws Exception {
+        LOG.info("Test one script with updated class");
+
+        File testFile = File.createTempFile("script", ".py");
+        File original = new File("target/test-scripts/readTest.py");
+        FileUtils.copyFile(original, testFile);
+        controller.setProperty(ExecuteScript.SCRIPT_FILE_NAME, testFile.getPath());
+        controller.assertValid();
+        original = new File("target/test-scripts/readWithParams.py");
+        FileUtils.copyFile(original, testFile);
+        controller.setProperty(ExecuteScript.SCRIPT_CHECK_INTERVAL, "5 secs");
+        Thread.sleep(6000);
+
+        controller.assertNotValid(); // need to set 'expr'
+        controller.setProperty("int", "abc");
+        controller.assertNotValid();
+        controller.setProperty("url", "not@valid");
+        controller.assertNotValid();
+        controller.setProperty("nonEmpty", "");
+        controller.assertNotValid();
+
+        controller.setProperty("expr", "sed do");
+        controller.assertValid();
+        assertEquals(6, controller.getProcessContext().getProperties().size());
+        FileUtils.deleteQuietly(testFile);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/alwaysFail.js
----------------------------------------------------------------------
diff --git a/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/alwaysFail.js b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/alwaysFail.js
new file mode 100644
index 0000000..40f038d
--- /dev/null
+++ b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/alwaysFail.js
@@ -0,0 +1,24 @@
+/*
+ * 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.
+ */
+with (Scripting) {
+    var instance = new ReaderScript({
+        validate: function () {
+            return ["This will never work."];
+        }
+    });
+}
+

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/alwaysFail.py
----------------------------------------------------------------------
diff --git a/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/alwaysFail.py b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/alwaysFail.py
new file mode 100644
index 0000000..4b744ab
--- /dev/null
+++ b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/alwaysFail.py
@@ -0,0 +1,19 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+# type(name, superclass_tuple, member_dict) is a shorthand for defining an
+# anonymous class.  Note the trailing parens (), because scriptBuilder must
+# return an *instance* of the class.
+instance = type("FailingReader", (ReaderScript, object),\
+        {"validate": lambda self : ["I am broken"]})()
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/alwaysFail.rb
----------------------------------------------------------------------
diff --git a/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/alwaysFail.rb b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/alwaysFail.rb
new file mode 100644
index 0000000..076cdfa
--- /dev/null
+++ b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/alwaysFail.rb
@@ -0,0 +1,21 @@
+# 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.
+class NeverSatisfied < ReaderScript
+  def validate
+    return ["This is supposed to fail"]
+  end
+end
+
+NeverSatisfied.new

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/ffTest.js
----------------------------------------------------------------------
diff --git a/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/ffTest.js b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/ffTest.js
new file mode 100644
index 0000000..d40623e
--- /dev/null
+++ b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/ffTest.js
@@ -0,0 +1,28 @@
+/*
+ * 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.
+ */
+with (Scripting) {
+    var instance = new ReaderScript({
+        route: function (input) {
+            if (instance.getFlowFileSize() < 20) {
+                return Script.FAIL_RELATIONSHIP;
+            } else {
+                return Script.SUCCESS_RELATIONSHIP;
+            }
+        }
+    });
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/ffTest.py
----------------------------------------------------------------------
diff --git a/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/ffTest.py b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/ffTest.py
new file mode 100644
index 0000000..f4deb22
--- /dev/null
+++ b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/ffTest.py
@@ -0,0 +1,22 @@
+# 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.
+class SimpleJythonReader(ReaderScript):
+    def route(self, input):
+        if self.getFlowFileSize() < 20 : return self.FAIL_RELATIONSHIP
+             
+        return self.SUCCESS_RELATIONSHIP
+
+
+instance = SimpleJythonReader()

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/ffTest.rb
----------------------------------------------------------------------
diff --git a/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/ffTest.rb b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/ffTest.rb
new file mode 100644
index 0000000..0435ea7
--- /dev/null
+++ b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/ffTest.rb
@@ -0,0 +1,30 @@
+# 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.
+class SimpleJRubyReader < ReaderScript
+  field_reader :FAIL_RELATIONSHIP, :SUCCESS_RELATIONSHIP
+  
+  def getRelationships 
+    @@evict = Relationship::Builder.new.name("evict").description("some evicted stuff").build()
+    [FAIL_RELATIONSHIP, SUCCESS_RELATIONSHIP, @@evict]
+  end
+  
+  def route( input )
+    return FAIL_RELATIONSHIP if getFlowFileSize < 20
+	return @@evict if !getAttribute("evict").nil?
+	return SUCCESS_RELATIONSHIP
+	end
+end
+
+SimpleJRubyReader.new

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/lib/Sub.py
----------------------------------------------------------------------
diff --git a/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/lib/Sub.py b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/lib/Sub.py
new file mode 100644
index 0000000..827816c
--- /dev/null
+++ b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/lib/Sub.py
@@ -0,0 +1,18 @@
+# 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.
+class TestFilter:
+    def notAllowed(self):
+        return "^sed"
+ 
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/lib/sub.js
----------------------------------------------------------------------
diff --git a/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/lib/sub.js b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/lib/sub.js
new file mode 100644
index 0000000..489036a
--- /dev/null
+++ b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/lib/sub.js
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+function notAllowed() {  // Works for eval(readFile(...))
+    return /sed do/i;
+}
+
+exports.notAllowed = notAllowed;  // Works for require(...)
+

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/lib/sub.rb
----------------------------------------------------------------------
diff --git a/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/lib/sub.rb b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/lib/sub.rb
new file mode 100644
index 0000000..775aad4
--- /dev/null
+++ b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/lib/sub.rb
@@ -0,0 +1,17 @@
+# 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.
+def notAllowed
+  return /^sed/i
+end
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/loadLocal.js
----------------------------------------------------------------------
diff --git a/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/loadLocal.js b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/loadLocal.js
new file mode 100644
index 0000000..39396ab
--- /dev/null
+++ b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/loadLocal.js
@@ -0,0 +1,30 @@
+/*
+ * 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.
+ */
+var sub = require("lib/sub.js");
+
+with (Scripting) {
+    var instance = new ReaderScript({
+        route: function (input) {
+            var str = IOUtils.toString(input);
+            if (str.match(sub.notAllowed())) {
+                return Script.FAIL_RELATIONSHIP;
+            } else {
+                return Script.SUCCESS_RELATIONSHIP;
+            }
+        }
+    });
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/loadLocal.py
----------------------------------------------------------------------
diff --git a/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/loadLocal.py b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/loadLocal.py
new file mode 100644
index 0000000..2429ea2
--- /dev/null
+++ b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/loadLocal.py
@@ -0,0 +1,26 @@
+# 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.
+import re
+from Sub import TestFilter
+class SimpleJythonReader(ReaderScript):
+    def route(self, input):
+        tf = TestFilter()
+        for line in FileUtil.wrap(input):
+            if re.match(tf.notAllowed(),line): return self.FAIL_RELATIONSHIP
+             
+        return self.SUCCESS_RELATIONSHIP
+
+
+instance = SimpleJythonReader()

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/loadLocal.rb
----------------------------------------------------------------------
diff --git a/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/loadLocal.rb b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/loadLocal.rb
new file mode 100644
index 0000000..f1b3ec5
--- /dev/null
+++ b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/loadLocal.rb
@@ -0,0 +1,29 @@
+# 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.
+require 'sub'
+
+class SimpleJRubyReader < ReaderScript
+  field_reader :FAIL_RELATIONSHIP, :SUCCESS_RELATIONSHIP
+  
+  def route( input )
+    input.to_io.each_line do |line|
+      return FAIL_RELATIONSHIP if line.match notAllowed
+    end
+
+    return SUCCESS_RELATIONSHIP
+  end
+end
+
+SimpleJRubyReader.new

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/log4j.xml
----------------------------------------------------------------------
diff --git a/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/log4j.xml b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/log4j.xml
new file mode 100644
index 0000000..d984dff
--- /dev/null
+++ b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/log4j.xml
@@ -0,0 +1,54 @@
+<?xml version="1.0" encoding="UTF-8" ?>
+<!--
+  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.
+-->
+<!DOCTYPE log4j:configuration SYSTEM "log4j.dtd">
+
+<log4j:configuration xmlns:log4j="http://jakarta.apache.org/log4j/">
+    <appender name="CONSOLE" class="org.apache.log4j.ConsoleAppender">
+        <param name="Target" value="System.out"/>
+        <param name="Threshold" value="DEBUG"/>
+
+        <layout class="org.apache.log4j.PatternLayout">
+            <param name="ConversionPattern" value="%d{MM-dd-yy HH:mm:ss.SSS} %-5p %c{2} %x - %m%n"/>
+        </layout>
+
+    </appender>
+
+    <logger name="org.nifi.model.processor.FlowFileProcessor" additivity="false">
+        <level value="INFO"/>
+        <appender-ref ref="CONSOLE"/>
+    </logger>
+
+    <logger name="junit.TestableAppender" additivity="false">
+        <level value="TRACE"/>
+        <appender-ref ref="CONSOLE"/>
+    </logger>
+
+    <logger name="org.nifi.model.processor.impl.ScriptRunnerProcessor" additivity="false">
+        <level value="WARN"/>
+        <appender-ref ref="CONSOLE"/>
+    </logger>
+  
+    <logger name="org.nifi.model.processor.impl.ScriptRunnerProcessorTest" additivity="false">
+        <level value="INFO"/>
+        <appender-ref ref="CONSOLE"/>
+    </logger>
+
+    <root>
+        <level value="INFO"/>
+        <appender-ref ref="CONSOLE" />
+    </root>
+
+</log4j:configuration>

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/optionalValidators.js
----------------------------------------------------------------------
diff --git a/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/optionalValidators.js b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/optionalValidators.js
new file mode 100644
index 0000000..cf7c551
--- /dev/null
+++ b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/optionalValidators.js
@@ -0,0 +1,28 @@
+/*
+ * 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.
+ */
+with (Scripting) {
+    var instance = new WriterScript({
+        getPropertyDescriptors: function () {
+            i = new PropertyDescriptor.Builder().name("int").description("an int").required(true).addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR).build();
+            u = new PropertyDescriptor.Builder().name("url").description("a url").required(true).addValidator(StandardValidators.URL_VALIDATOR).build();
+            s = new PropertyDescriptor.Builder().name("nonEmpty").description("a non empty property").required(true).addValidator(StandardValidators.NON_EMPTY_VALIDATOR).build();
+            return [i, u, s];
+        }
+    });
+
+}
+

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/optionalValidators.py
----------------------------------------------------------------------
diff --git a/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/optionalValidators.py b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/optionalValidators.py
new file mode 100644
index 0000000..9f1a43d
--- /dev/null
+++ b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/optionalValidators.py
@@ -0,0 +1,22 @@
+# 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.
+class SimpleJythonReader(ReaderScript):
+    def getPropertyDescriptors( self ):
+        nev = PropertyDescriptor.Builder().name("nonEmpty").required(1).addValidator(StandardValidators.NON_EMPTY_VALIDATOR).build()
+        iv = PropertyDescriptor.Builder().name("int").required(1).addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR).build()
+        uv = PropertyDescriptor.Builder().name("url").required(1).addValidator(StandardValidators.URL_VALIDATOR).build()
+        return [nev, iv, uv]
+
+instance = SimpleJythonReader()

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/optionalValidators.rb
----------------------------------------------------------------------
diff --git a/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/optionalValidators.rb b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/optionalValidators.rb
new file mode 100644
index 0000000..3fa53b4
--- /dev/null
+++ b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/optionalValidators.rb
@@ -0,0 +1,39 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+class SimpleJRubyReader < ReaderScript
+  field_reader :FAIL_RELATIONSHIP, :SUCCESS_RELATIONSHIP, :logger
+  
+  def getPropertyDescriptors
+    logger.debug("Defining descriptors");
+    i = StandardValidators::INTEGER_VALIDATOR
+    u = StandardValidators::URL_VALIDATOR
+    s = StandardValidators::NON_EMPTY_VALIDATOR
+    intPropDesc = PropertyDescriptor::Builder.new().name("int").required(true).addValidator(i).build()
+    urlPropDesc = PropertyDescriptor::Builder.new().name("url").required(true).addValidator(u).build()
+    nonEmptyPropDesc = PropertyDescriptor::Builder.new().name("nonEmpty").addValidator(s).build()
+    return [intPropDesc, urlPropDesc, nonEmptyPropDesc]
+  end
+  
+  def route( input )
+    logger.debug("Routing input");
+    input.to_io.each_line do |line|
+      return FAIL_RELATIONSHIP if line.match /^sed/i
+    end
+
+    return SUCCESS_RELATIONSHIP
+	end
+end
+$logger.debug("Creating SimpleJRubyReader with props" + @properties.to_s)
+SimpleJRubyReader.new
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/paramTest.js
----------------------------------------------------------------------
diff --git a/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/paramTest.js b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/paramTest.js
new file mode 100644
index 0000000..c00537a
--- /dev/null
+++ b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/paramTest.js
@@ -0,0 +1,28 @@
+/*
+ * 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.
+ */
+with (Scripting) {
+    var instance = new WriterScript({
+        process: function (input, output) {
+            var str = IOUtils.toString(input);
+            var last = str.split("\n").pop() + "\n";
+            for (var i = 0; i < instance.getProperty("repeat"); i++) {
+                IOUtils.write(last, output);
+            }
+            output.flush();
+        }
+    });
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/paramTest.py
----------------------------------------------------------------------
diff --git a/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/paramTest.py b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/paramTest.py
new file mode 100644
index 0000000..3495e4a
--- /dev/null
+++ b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/paramTest.py
@@ -0,0 +1,26 @@
+# 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.
+class ParamUsingWriter(WriterScript):
+    def process ( self, input, output ):
+        last = FileUtil.wrap(input).readlines()[-1] + '\n'
+        writer = FileUtil.wrap(output)
+        times = int(self.getProperty("repeat"))
+        lines = [last] * times
+
+        writer.writelines(lines)
+
+        writer.close()
+
+instance = ParamUsingWriter()

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/paramTest.rb
----------------------------------------------------------------------
diff --git a/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/paramTest.rb b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/paramTest.rb
new file mode 100644
index 0000000..7c2eb23
--- /dev/null
+++ b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/paramTest.rb
@@ -0,0 +1,31 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+class ParamUsingWriter < WriterScript
+  def process ( input, output )
+    reader = input.to_io
+    writer = output.to_io
+    
+    last = reader.readlines.last
+    
+    getProperty("repeat").to_i.times do
+      writer << last + "\n"
+    end
+    
+    writer.close
+    reader.close
+  end
+end
+
+ParamUsingWriter.new

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/4d998c12/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/parseXml.js
----------------------------------------------------------------------
diff --git a/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/parseXml.js b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/parseXml.js
new file mode 100644
index 0000000..901e158
--- /dev/null
+++ b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/parseXml.js
@@ -0,0 +1,36 @@
+/*
+ * 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.
+ */
+with (Scripting) {
+    var instance = new WriterScript({
+        process: function (istream, ostream) {
+            var str = IOUtils.toString(istream);
+            var obj = new XML(str);
+            print(obj)
+            var good = obj.b.(@good == "true");
+                    if (good.length() == 0) {
+                instance.setRoute(Script.FAIL_RELATIONSHIP);
+                IOUtils.write(str, ostream);
+            } else {
+                instance.setRoute(Script.SUCCESS_RELATIONSHIP);
+                for each (var goodStr in good) {
+                    IOUtils.write(goodStr, ostream);
+                }
+            }
+            ostream.flush();
+        }
+    });
+}
\ No newline at end of file