From 77ab5d36843370a30141f660c0b4dd146b007ff7 Mon Sep 17 00:00:00 2001 From: Matt Burgess Date: Mon, 1 Feb 2016 19:13:30 -0500 Subject: [PATCH] NIFI-210: This closes #185. Add ExecuteScript and InvokeScriptedProcessor Signed-off-by: joewitt --- nifi-assembly/LICENSE | 23 + nifi-assembly/NOTICE | 36 ++ nifi-assembly/pom.xml | 5 + .../nifi-scripting-nar/pom.xml | 35 ++ .../src/main/resources/META-INF/LICENSE | 274 +++++++++ .../src/main/resources/META-INF/NOTICE | 47 ++ .../nifi-scripting-processors/pom.xml | 72 +++ .../script/AbstractScriptProcessor.java | 302 ++++++++++ .../nifi/processors/script/ExecuteScript.java | 213 +++++++ .../script/InvokeScriptedProcessor.java | 554 ++++++++++++++++++ .../script/ScriptEngineConfigurator.java | 35 ++ .../impl/GroovyScriptEngineConfigurator.java | 55 ++ .../impl/JythonScriptEngineConfigurator.java | 51 ++ .../org.apache.nifi.processor.Processor | 16 + ...processors.script.ScriptEngineConfigurator | 17 + .../processors/script/BaseScriptTest.java | 72 +++ .../processors/script/TestExecuteGroovy.java | 291 +++++++++ .../processors/script/TestExecuteJRuby.java | 59 ++ .../script/TestExecuteJavascript.java | 58 ++ .../processors/script/TestExecuteJython.java | 77 +++ .../processors/script/TestExecuteLua.java | 59 ++ .../processors/script/TestInvokeGroovy.java | 171 ++++++ .../processors/script/TestInvokeJython.java | 169 ++++++ ...estCreateNewFlowFileWithNoInputFile.groovy | 20 + .../testInvokeScriptCausesException.groovy | 62 ++ ...reInFlowFileAttributeWithScriptBody.groovy | 23 + ...ileAttributeWithScriptBodyNoModules.groovy | 20 + ...ntAndStoreInFlowFileCustomAttribute.groovy | 20 + .../groovy/testScriptNoTransfer.groovy | 19 + .../groovy/testScriptRoutesToFailure.groovy | 64 ++ .../groovy/test_dynamicProperties.groovy | 22 + .../resources/groovy/test_onTrigger.groovy | 24 + .../test_onTrigger_changeContent.groovy | 41 ++ .../groovy/test_onTrigger_newFlowFile.groovy | 46 ++ .../test/resources/groovy/test_reader.groovy | 77 +++ .../resources/javascript/test_onTrigger.js | 21 + .../test/resources/jruby/test_onTrigger.rb | 22 + .../resources/jython/callbacks/__init__.py | 23 + .../resources/jython/callbacks/compress.py | 49 ++ .../resources/jython/callbacks/decompress.py | 48 ++ .../jython/callbacks/read_first_line.py | 50 ++ .../test/resources/jython/test_compress.py | 76 +++ .../src/test/resources/jython/test_invalid.py | 48 ++ .../src/test/resources/jython/test_reader.py | 74 +++ .../resources/jython/test_update_attribute.py | 79 +++ .../src/test/resources/lua/test_onTrigger.lua | 21 + .../nifi-scripting-bundle/pom.xml | 43 ++ nifi-nar-bundles/pom.xml | 1 + pom.xml | 14 +- 49 files changed, 3694 insertions(+), 4 deletions(-) create mode 100644 nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-nar/pom.xml create mode 100644 nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-nar/src/main/resources/META-INF/LICENSE create mode 100644 nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-nar/src/main/resources/META-INF/NOTICE create mode 100644 nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/pom.xml create mode 100644 nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/AbstractScriptProcessor.java create mode 100644 nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/ExecuteScript.java create mode 100644 nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/InvokeScriptedProcessor.java create mode 100644 nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/ScriptEngineConfigurator.java create mode 100644 nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/impl/GroovyScriptEngineConfigurator.java create mode 100644 nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/impl/JythonScriptEngineConfigurator.java create mode 100644 nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor create mode 100644 nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/resources/META-INF/services/org.apache.nifi.processors.script.ScriptEngineConfigurator create mode 100644 nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/java/org/apache/nifi/processors/script/BaseScriptTest.java create mode 100644 nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/java/org/apache/nifi/processors/script/TestExecuteGroovy.java create mode 100644 nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/java/org/apache/nifi/processors/script/TestExecuteJRuby.java create mode 100644 nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/java/org/apache/nifi/processors/script/TestExecuteJavascript.java create mode 100644 nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/java/org/apache/nifi/processors/script/TestExecuteJython.java create mode 100644 nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/java/org/apache/nifi/processors/script/TestExecuteLua.java create mode 100644 nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/java/org/apache/nifi/processors/script/TestInvokeGroovy.java create mode 100755 nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/java/org/apache/nifi/processors/script/TestInvokeJython.java create mode 100644 nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/resources/groovy/testCreateNewFlowFileWithNoInputFile.groovy create mode 100644 nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/resources/groovy/testInvokeScriptCausesException.groovy create mode 100644 nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/resources/groovy/testReadFlowFileContentAndStoreInFlowFileAttributeWithScriptBody.groovy create mode 100644 nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/resources/groovy/testReadFlowFileContentAndStoreInFlowFileAttributeWithScriptBodyNoModules.groovy create mode 100644 nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/resources/groovy/testReadFlowFileContentAndStoreInFlowFileCustomAttribute.groovy create mode 100644 nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/resources/groovy/testScriptNoTransfer.groovy create mode 100644 nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/resources/groovy/testScriptRoutesToFailure.groovy create mode 100644 nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/resources/groovy/test_dynamicProperties.groovy create mode 100644 nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/resources/groovy/test_onTrigger.groovy create mode 100644 nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/resources/groovy/test_onTrigger_changeContent.groovy create mode 100644 nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/resources/groovy/test_onTrigger_newFlowFile.groovy create mode 100644 nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/resources/groovy/test_reader.groovy create mode 100644 nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/resources/javascript/test_onTrigger.js create mode 100644 nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/resources/jruby/test_onTrigger.rb create mode 100755 nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/resources/jython/callbacks/__init__.py create mode 100755 nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/resources/jython/callbacks/compress.py create mode 100755 nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/resources/jython/callbacks/decompress.py create mode 100755 nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/resources/jython/callbacks/read_first_line.py create mode 100755 nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/resources/jython/test_compress.py create mode 100755 nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/resources/jython/test_invalid.py create mode 100755 nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/resources/jython/test_reader.py create mode 100755 nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/resources/jython/test_update_attribute.py create mode 100644 nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/resources/lua/test_onTrigger.lua create mode 100644 nifi-nar-bundles/nifi-scripting-bundle/pom.xml diff --git a/nifi-assembly/LICENSE b/nifi-assembly/LICENSE index ca4a8805a0..6aef7ed596 100644 --- a/nifi-assembly/LICENSE +++ b/nifi-assembly/LICENSE @@ -1104,6 +1104,29 @@ information can be found here: http://jsoup.org/license The above copyright notice and this permission notice shall be included in all copies or substantial portions of the Software. + THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR + IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, + FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE + AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER + LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, + OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN + THE SOFTWARE. + +This product bundles 'Luaj' which is available under an MIT style license. More +information can be found here: + + Copyright (c) 2009 Luaj.org. All rights reserved. + + Permission is hereby granted, free of charge, to any person obtaining a copy + of this software and associated documentation files (the "Software"), to deal + in the Software without restriction, including without limitation the rights + to use, copy, modify, merge, publish, distribute, sublicense, and/or sell + copies of the Software, and to permit persons to whom the Software is + furnished to do so, subject to the following conditions: + + The above copyright notice and this permission notice shall be included in + all copies or substantial portions of the Software. + THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE diff --git a/nifi-assembly/NOTICE b/nifi-assembly/NOTICE index 0a8c2d7e02..ac2e04c9c9 100644 --- a/nifi-assembly/NOTICE +++ b/nifi-assembly/NOTICE @@ -780,6 +780,16 @@ The following binary components are provided under the Apache Software License v (ASLv2) EventHubs Client (com.microsoft.eventhubs.client:eventhubs-client:0.9.1 - https://github.com/hdinsight/eventhubs-client/) + (ASLv2) Groovy (org.codehaus.groovy:groovy:jar:2.4.5 - http://www.groovy-lang.org) + The following NOTICE information applies: + Groovy Language + Copyright 2003-2015 The respective authors and developers + Developers and Contributors are listed in the project POM file + and Gradle build file + + This product includes software developed by + The Groovy community (http://groovy.codehaus.org/). + ************************ Common Development and Distribution License 1.1 @@ -851,6 +861,12 @@ The following binary components are provided under the Eclipse Public License 1. (EPL 1.0)(MPL 2.0) H2 Database (com.h2database:h2:jar:1.3.176 - http://www.h2database.com/html/license.html) (EPL 1.0)(LGPL 2.1) Logback Classic (ch.qos.logback:logback-classic:jar:1.1.3 - http://logback.qos.ch/) (EPL 1.0)(LGPL 2.1) Logback Core (ch.qos.logback:logback-core:jar:1.1.3 - http://logback.qos.ch/) + (EPLv1.0) JRuby (org.jruby:jruby-complete:9.0.4.0 - http://jruby.org). + + JRuby is licensed under three licenses - the EPL 1.0, GPL 2 and LGPL 2.1. Apache NiFi uses the EPL v1.0 license. + + The following NOTICE information applies: + Copyright (c) 2007-2015 The JRuby project ***************** Mozilla Public License v2.0 @@ -869,6 +885,26 @@ The following binary components are provided under the Mozilla Public License v1 (MPL 1.1) HAPI Base (ca.uhn.hapi:hapi-base:2.2 - http://hl7api.sourceforge.net/) (MPL 1.1) HAPI Structures (ca.uhn.hapi:hapi-structures-v*:2.2 - http://hl7api.sourceforge.net/) +****************** +Python Software Foundation License v2 +****************** + +The following binary components are provided under the Python Software Foundation License v2 + + (PSFLv2) Jython (org.python:jython-standalone:2.7.0 - http://www.jython.org/) + The following NOTICE information applies: + Copyright (c) 2000, 2001, 2002, 2003, 2004, 2005, 2006, 2007 Jython Developers All rights reserved. + +****************** +MIT License +****************** + +The following binary components are provided under an MIT-style license + + (MIT) Luaj (org.luaj:luaj-jse:3.0.1 - http://www.luaj.org/luaj/3.0/README.html) + The following NOTICE information applies: + Copyright (c) 2009-2011 Luaj.org. All rights reserved. + ***************** Public Domain ***************** diff --git a/nifi-assembly/pom.xml b/nifi-assembly/pom.xml index f5e9a984fc..0bf722b121 100644 --- a/nifi-assembly/pom.xml +++ b/nifi-assembly/pom.xml @@ -262,6 +262,11 @@ language governing permissions and limitations under the License. --> nifi-azure-nar nar + + org.apache.nifi + nifi-scripting-nar + nar + diff --git a/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-nar/pom.xml b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-nar/pom.xml new file mode 100644 index 0000000000..89cbc4f6d0 --- /dev/null +++ b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-nar/pom.xml @@ -0,0 +1,35 @@ + + + + 4.0.0 + + + org.apache.nifi + nifi-scripting-bundle + 0.4.2-SNAPSHOT + + + nifi-scripting-nar + nar + + + + org.apache.nifi + nifi-scripting-processors + + + + diff --git a/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-nar/src/main/resources/META-INF/LICENSE b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-nar/src/main/resources/META-INF/LICENSE new file mode 100644 index 0000000000..671d529c28 --- /dev/null +++ b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-nar/src/main/resources/META-INF/LICENSE @@ -0,0 +1,274 @@ + + Apache License + Version 2.0, January 2004 + http://www.apache.org/licenses/ + + TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION + + 1. Definitions. + + "License" shall mean the terms and conditions for use, reproduction, + and distribution as defined by Sections 1 through 9 of this document. + + "Licensor" shall mean the copyright owner or entity authorized by + the copyright owner that is granting the License. + + "Legal Entity" shall mean the union of the acting entity and all + other entities that control, are controlled by, or are under common + control with that entity. For the purposes of this definition, + "control" means (i) the power, direct or indirect, to cause the + direction or management of such entity, whether by contract or + otherwise, or (ii) ownership of fifty percent (50%) or more of the + outstanding shares, or (iii) beneficial ownership of such entity. + + "You" (or "Your") shall mean an individual or Legal Entity + exercising permissions granted by this License. + + "Source" form shall mean the preferred form for making modifications, + including but not limited to software source code, documentation + source, and configuration files. + + "Object" form shall mean any form resulting from mechanical + transformation or translation of a Source form, including but + not limited to compiled object code, generated documentation, + and conversions to other media types. + + "Work" shall mean the work of authorship, whether in Source or + Object form, made available under the License, as indicated by a + copyright notice that is included in or attached to the work + (an example is provided in the Appendix below). + + "Derivative Works" shall mean any work, whether in Source or Object + form, that is based on (or derived from) the Work and for which the + editorial revisions, annotations, elaborations, or other modifications + represent, as a whole, an original work of authorship. For the purposes + of this License, Derivative Works shall not include works that remain + separable from, or merely link (or bind by name) to the interfaces of, + the Work and Derivative Works thereof. + + "Contribution" shall mean any work of authorship, including + the original version of the Work and any modifications or additions + to that Work or Derivative Works thereof, that is intentionally + submitted to Licensor for inclusion in the Work by the copyright owner + or by an individual or Legal Entity authorized to submit on behalf of + the copyright owner. For the purposes of this definition, "submitted" + means any form of electronic, verbal, or written communication sent + to the Licensor or its representatives, including but not limited to + communication on electronic mailing lists, source code control systems, + and issue tracking systems that are managed by, or on behalf of, the + Licensor for the purpose of discussing and improving the Work, but + excluding communication that is conspicuously marked or otherwise + designated in writing by the copyright owner as "Not a Contribution." + + "Contributor" shall mean Licensor and any individual or Legal Entity + on behalf of whom a Contribution has been received by Licensor and + subsequently incorporated within the Work. + + 2. Grant of Copyright License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + copyright license to reproduce, prepare Derivative Works of, + publicly display, publicly perform, sublicense, and distribute the + Work and such Derivative Works in Source or Object form. + + 3. Grant of Patent License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + (except as stated in this section) patent license to make, have made, + use, offer to sell, sell, import, and otherwise transfer the Work, + where such license applies only to those patent claims licensable + by such Contributor that are necessarily infringed by their + Contribution(s) alone or by combination of their Contribution(s) + with the Work to which such Contribution(s) was submitted. If You + institute patent litigation against any entity (including a + cross-claim or counterclaim in a lawsuit) alleging that the Work + or a Contribution incorporated within the Work constitutes direct + or contributory patent infringement, then any patent licenses + granted to You under this License for that Work shall terminate + as of the date such litigation is filed. + + 4. Redistribution. You may reproduce and distribute copies of the + Work or Derivative Works thereof in any medium, with or without + modifications, and in Source or Object form, provided that You + meet the following conditions: + + (a) You must give any other recipients of the Work or + Derivative Works a copy of this License; and + + (b) You must cause any modified files to carry prominent notices + stating that You changed the files; and + + (c) You must retain, in the Source form of any Derivative Works + that You distribute, all copyright, patent, trademark, and + attribution notices from the Source form of the Work, + excluding those notices that do not pertain to any part of + the Derivative Works; and + + (d) If the Work includes a "NOTICE" text file as part of its + distribution, then any Derivative Works that You distribute must + include a readable copy of the attribution notices contained + within such NOTICE file, excluding those notices that do not + pertain to any part of the Derivative Works, in at least one + of the following places: within a NOTICE text file distributed + as part of the Derivative Works; within the Source form or + documentation, if provided along with the Derivative Works; or, + within a display generated by the Derivative Works, if and + wherever such third-party notices normally appear. The contents + of the NOTICE file are for informational purposes only and + do not modify the License. You may add Your own attribution + notices within Derivative Works that You distribute, alongside + or as an addendum to the NOTICE text from the Work, provided + that such additional attribution notices cannot be construed + as modifying the License. + + You may add Your own copyright statement to Your modifications and + may provide additional or different license terms and conditions + for use, reproduction, or distribution of Your modifications, or + for any such Derivative Works as a whole, provided Your use, + reproduction, and distribution of the Work otherwise complies with + the conditions stated in this License. + + 5. Submission of Contributions. Unless You explicitly state otherwise, + any Contribution intentionally submitted for inclusion in the Work + by You to the Licensor shall be under the terms and conditions of + this License, without any additional terms or conditions. + Notwithstanding the above, nothing herein shall supersede or modify + the terms of any separate license agreement you may have executed + with Licensor regarding such Contributions. + + 6. Trademarks. This License does not grant permission to use the trade + names, trademarks, service marks, or product names of the Licensor, + except as required for reasonable and customary use in describing the + origin of the Work and reproducing the content of the NOTICE file. + + 7. Disclaimer of Warranty. Unless required by applicable law or + agreed to in writing, Licensor provides the Work (and each + Contributor provides its Contributions) on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + implied, including, without limitation, any warranties or conditions + of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A + PARTICULAR PURPOSE. You are solely responsible for determining the + appropriateness of using or redistributing the Work and assume any + risks associated with Your exercise of permissions under this License. + + 8. Limitation of Liability. In no event and under no legal theory, + whether in tort (including negligence), contract, or otherwise, + unless required by applicable law (such as deliberate and grossly + negligent acts) or agreed to in writing, shall any Contributor be + liable to You for damages, including any direct, indirect, special, + incidental, or consequential damages of any character arising as a + result of this License or out of the use or inability to use the + Work (including but not limited to damages for loss of goodwill, + work stoppage, computer failure or malfunction, or any and all + other commercial damages or losses), even if such Contributor + has been advised of the possibility of such damages. + + 9. Accepting Warranty or Additional Liability. While redistributing + the Work or Derivative Works thereof, You may choose to offer, + and charge a fee for, acceptance of support, warranty, indemnity, + or other liability obligations and/or rights consistent with this + License. However, in accepting such obligations, You may act only + on Your own behalf and on Your sole responsibility, not on behalf + of any other Contributor, and only if You agree to indemnify, + defend, and hold each Contributor harmless for any liability + incurred by, or claims asserted against, such Contributor by reason + of your accepting any such warranty or additional liability. + + END OF TERMS AND CONDITIONS + + APPENDIX: How to apply the Apache License to your work. + + To apply the Apache License to your work, attach the following + boilerplate notice, with the fields enclosed by brackets "[]" + replaced with your own identifying information. (Don't include + the brackets!) The text should be enclosed in the appropriate + comment syntax for the file format. We also recommend that a + file or class name and description of purpose be included on the + same "printed page" as the copyright notice for easier + identification within third-party archives. + + Copyright [yyyy] [name of copyright owner] + + Licensed 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. + +APACHE NIFI SUBCOMPONENTS: + +The Apache NiFi project contains subcomponents with separate copyright +notices and license terms. Your use of the source code for the these +subcomponents is subject to the terms and conditions of the following +licenses. + + The binary distribution of this product bundles 'Jython standalone' under a Python Software Foundation license + + Copyright (c) 2000, 2001, 2002, 2003, 2004, 2005, 2006, 2007 Jython Developers All rights reserved. + Copyright (c) 2007 Python Software Foundation; All Rights Reserved + + PYTHON SOFTWARE FOUNDATION LICENSE VERSION 2 + + 1. This LICENSE AGREEMENT is between the Python Software Foundation ("PSF"), and the Individual or Organization + ("Licensee") accessing and otherwise using this software ("Jython") in source or binary form and its associated + documentation. + + 2. Subject to the terms and conditions of this License Agreement, PSF hereby grants Licensee a nonexclusive, + royalty-free, world-wide license to reproduce, analyze, test, perform and/or display publicly, prepare derivative + works, distribute, and otherwise use Jython alone or in any derivative version, provided, however, that PSF's + License Agreement and PSF's notice of copyright, i.e., + "Copyright (c) 2007 Python Software Foundation; All Rights Reserved" are retained in Jython alone or in any + derivative version prepared by Licensee. + + 3. In the event Licensee prepares a derivative work that is based on or incorporates Jython or any part thereof, + and wants to make the derivative work available to others as provided herein, then Licensee hereby agrees to + include in any such work a brief summary of the changes made to Jython. + + 4. PSF is making Jython available to Licensee on an "AS IS" basis. PSF MAKES NO REPRESENTATIONS OR WARRANTIES, + EXPRESS OR IMPLIED. BY WAY OF EXAMPLE, BUT NOT LIMITATION, PSF MAKES NO AND DISCLAIMS ANY REPRESENTATION OR + WARRANTY OF MERCHANTABILITY OR FITNESS FOR ANY PARTICULAR PURPOSE OR THAT THE USE OF JYTHON WILL NOT INFRINGE + ANY THIRD PARTY RIGHTS. + + 5. PSF SHALL NOT BE LIABLE TO LICENSEE OR ANY OTHER USERS OF JYTHON FOR ANY INCIDENTAL, SPECIAL, OR CONSEQUENTIAL + DAMAGES OR LOSS AS A RESULT OF MODIFYING, DISTRIBUTING, OR OTHERWISE USING JYTHON, OR ANY DERIVATIVE THEREOF, EVEN + IF ADVISED OF THE POSSIBILITY THEREOF. + + 6. This License Agreement will automatically terminate upon a material breach of its terms and conditions. + + 7. Nothing in this License Agreement shall be deemed to create any relationship of agency, partnership, or joint + venture between PSF and Licensee. This License Agreement does not grant permission to use PSF trademarks or trade + name in a trademark sense to endorse or promote products or services of Licensee, or any third party. + + 8. By copying, installing or otherwise using Jython, Licensee agrees to be bound by the terms and conditions of + this License Agreement. + + + The binary distribution of this product bundles 'Luaj' under an MIT + style license. + + Copyright © 2009-2014 Luaj.org. + + Permission is hereby granted, free of charge, to any person obtaining + a copy of this software and associated documentation files (the + "Software"), to deal in the Software without restriction, including + without limitation the rights to use, copy, modify, merge, publish, + distribute, sublicense, and/or sell copies of the Software, and to + permit persons to whom the Software is furnished to do so, subject to + the following conditions: + + The above copyright notice and this permission notice shall be + included in all copies or substantial portions of the Software. + + THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, + EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF + MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND + NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE + LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION + OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION + WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. \ No newline at end of file diff --git a/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-nar/src/main/resources/META-INF/NOTICE b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-nar/src/main/resources/META-INF/NOTICE new file mode 100644 index 0000000000..2e09e083cc --- /dev/null +++ b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-nar/src/main/resources/META-INF/NOTICE @@ -0,0 +1,47 @@ +nifi-scripting-nar +Copyright 2015 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + +****************** +Apache Software License v2 +****************** + +The following binary components are provided under the Apache Software License v2 + + (ASLv2) Groovy (org.codehaus.groovy:groovy:jar:2.4.5 - http://www.groovy-lang.org) + The following NOTICE information applies: + Groovy Language + Copyright 2003-2015 The respective authors and developers + Developers and Contributors are listed in the project POM file + and Gradle build file + + This product includes software developed by + The Groovy community (http://groovy.codehaus.org/). + + (ASLv2) Apache Commons IO + The following NOTICE information applies: + Apache Commons IO + Copyright 2002-2012 The Apache Software Foundation + + (ASLv2) Apache Commons Lang + The following NOTICE information applies: + Apache Commons Lang + Copyright 2001-2015 The Apache Software Foundation + + This product includes software from the Spring Framework, + under the Apache License 2.0 (see: StringUtils.containsWhitespace()) + +****************** +Eclipse Public License v1.0 +****************** + +The following binary components are provided under a EPL v1.0 license + + (EPLv1.0) JRuby (org.jruby:jruby-complete:9.0.4.0 - http://jruby.org). + + JRuby is licensed under three licenses - the EPL 1.0, GPL 2 and LGPL 2.1. Apache NiFi uses the EPL v1.0 license. + + The following NOTICE information applies: + Copyright (c) 2007-2015 The JRuby project \ No newline at end of file diff --git a/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/pom.xml b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/pom.xml new file mode 100644 index 0000000000..0f210f0c8f --- /dev/null +++ b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/pom.xml @@ -0,0 +1,72 @@ + + + + 4.0.0 + + + org.apache.nifi + nifi-scripting-bundle + 0.4.2-SNAPSHOT + + + nifi-scripting-processors + jar + + + + org.apache.nifi + nifi-api + + + org.apache.nifi + nifi-properties + + + org.apache.nifi + nifi-processor-utils + + + org.codehaus.groovy + groovy-all + 2.4.5 + + + org.python + jython-standalone + 2.7.0 + + + org.luaj + luaj-jse + 3.0.1 + + + org.jruby + jruby-complete + 9.0.4.0 + + + commons-io + commons-io + + + org.apache.nifi + nifi-mock + test + + + diff --git a/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/AbstractScriptProcessor.java b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/AbstractScriptProcessor.java new file mode 100644 index 0000000000..61fffa8d8e --- /dev/null +++ b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/AbstractScriptProcessor.java @@ -0,0 +1,302 @@ +/* + * 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.components.AllowableValue; +import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.components.ValidationContext; +import org.apache.nifi.components.ValidationResult; +import org.apache.nifi.components.Validator; +import org.apache.nifi.logging.ProcessorLog; +import org.apache.nifi.processor.AbstractSessionFactoryProcessor; +import org.apache.nifi.processor.Relationship; +import org.apache.nifi.processor.util.StandardValidators; +import org.apache.nifi.util.StringUtils; + +import javax.script.ScriptEngine; +import javax.script.ScriptEngineFactory; +import javax.script.ScriptEngineManager; +import javax.script.ScriptException; +import java.io.File; +import java.net.MalformedURLException; +import java.net.URL; +import java.net.URLClassLoader; +import java.nio.file.Files; +import java.nio.file.Paths; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.Comparator; +import java.util.HashMap; +import java.util.HashSet; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; +import java.util.ServiceLoader; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.atomic.AtomicBoolean; + +/** + * This class contains variables and methods common to scripting processors + */ +public abstract class AbstractScriptProcessor extends AbstractSessionFactoryProcessor { + + public static final Relationship REL_SUCCESS = new Relationship.Builder() + .name("success") + .description("FlowFiles that were successfully processed") + .build(); + + public static final Relationship REL_FAILURE = new Relationship.Builder() + .name("failure") + .description("FlowFiles that failed to be processed") + .build(); + + public static PropertyDescriptor SCRIPT_ENGINE; + + public static final PropertyDescriptor SCRIPT_FILE = new PropertyDescriptor.Builder() + .name("Script File") + .required(false) + .description("Path to script file to execute. Only one of Script File or Script Body may be used") + .addValidator(new StandardValidators.FileExistsValidator(true)) + .expressionLanguageSupported(true) + .build(); + + public static final PropertyDescriptor SCRIPT_BODY = new PropertyDescriptor.Builder() + .name("Script Body") + .required(false) + .description("Body of script to execute. Only one of Script File or Script Body may be used") + .addValidator(Validator.VALID) + .expressionLanguageSupported(false) + .build(); + + public static final PropertyDescriptor MODULES = new PropertyDescriptor.Builder() + .name("Module Directory") + .description("Path to a directory which contains modules required by the script.") + .required(false) + .expressionLanguageSupported(true) + .addValidator(new StandardValidators.DirectoryExistsValidator(true, false)) + .build(); + + // A map from engine name to a custom configurator for that engine + protected final Map scriptEngineConfiguratorMap = new ConcurrentHashMap<>(); + protected final AtomicBoolean isInitialized = new AtomicBoolean(false); + + protected Map scriptEngineFactoryMap; + protected String scriptEngineName; + protected String scriptPath; + protected String scriptBody; + protected String modulePath; + protected List descriptors; + protected ScriptEngine scriptEngine; + + /** + * Custom validation for ensuring exactly one of Script File or Script Body is populated + * + * @param validationContext provides a mechanism for obtaining externally + * managed values, such as property values and supplies convenience methods + * for operating on those values + * @return A collection of validation results + */ + @Override + protected Collection customValidate(ValidationContext validationContext) { + Set results = new HashSet<>(); + + // Verify that exactly one of "script file" or "script body" is set + Map propertyMap = validationContext.getProperties(); + if (StringUtils.isEmpty(propertyMap.get(SCRIPT_FILE)) == StringUtils.isEmpty(propertyMap.get(SCRIPT_BODY))) { + results.add(new ValidationResult.Builder().valid(false).explanation( + "Exactly one of Script File or Script Body must be set").build()); + } + + return results; + } + + /** + * This method creates all resources needed for the script processor to function, such as script engines, + * script file reloader threads, etc. + */ + protected void createResources() { + descriptors = new ArrayList<>(); + // The following is required for JRuby, should be transparent to everything else. + // Note this is not done in a ScriptEngineConfigurator, as it is too early in the lifecycle. The + // setting must be there before the factories/engines are loaded. + System.setProperty("org.jruby.embed.localvariable.behavior", "persistent"); + + // Create list of available engines + ScriptEngineManager scriptEngineManager = new ScriptEngineManager(); + List scriptEngineFactories = scriptEngineManager.getEngineFactories(); + if (scriptEngineFactories != null) { + scriptEngineFactoryMap = new HashMap<>(scriptEngineFactories.size()); + List engineList = new LinkedList<>(); + for (ScriptEngineFactory factory : scriptEngineFactories) { + engineList.add(new AllowableValue(factory.getLanguageName())); + scriptEngineFactoryMap.put(factory.getLanguageName(), factory); + } + + // Sort the list by name so the list always looks the same. + Collections.sort(engineList, new Comparator() { + @Override + public int compare(AllowableValue o1, AllowableValue o2) { + if (o1 == null) { + return o2 == null ? 0 : 1; + } + if (o2 == null) { + return -1; + } + return o1.getValue().compareTo(o2.getValue()); + } + }); + + AllowableValue[] engines = engineList.toArray(new AllowableValue[engineList.size()]); + + SCRIPT_ENGINE = new PropertyDescriptor.Builder() + .name("Script Engine") + .required(true) + .description("The engine to execute scripts") + .allowableValues(engines) + .defaultValue(engines[0].getValue()) + .required(true) + .expressionLanguageSupported(false) + .build(); + descriptors.add(SCRIPT_ENGINE); + } + + descriptors.add(SCRIPT_FILE); + descriptors.add(SCRIPT_BODY); + descriptors.add(MODULES); + + isInitialized.set(true); + } + + /** + * Determines whether the given path refers to a valid file + * + * @param path a path to a file + * @return true if the path refers to a valid file, false otherwise + */ + protected boolean isFile(final String path) { + return path != null && Files.isRegularFile(Paths.get(path)); + } + + /** + * Performs common setup operations when the processor is scheduled to run. This method assumes the member + * variables associated with properties have been filled. + * + */ + public void setup() { + + if (scriptEngineConfiguratorMap.isEmpty()) { + ServiceLoader configuratorServiceLoader = + ServiceLoader.load(ScriptEngineConfigurator.class); + for (ScriptEngineConfigurator configurator : configuratorServiceLoader) { + String configuratorScriptEngineName = configurator.getScriptEngineName(); + if (configuratorScriptEngineName != null + && configuratorScriptEngineName.equals(scriptEngineName)) { + scriptEngineConfiguratorMap.put(configurator.getScriptEngineName(), configurator); + } + } + } + setupEngine(); + } + + /** + * Configures the specified script engine. First, the engine is loaded and instantiated using the JSR-223 + * javax.script APIs. Then, if any script configurators have been defined for this engine, their init() method is + * called, and the configurator is saved for future calls. + * + * @see org.apache.nifi.processors.script.ScriptEngineConfigurator + */ + protected void setupEngine() { + ClassLoader originalContextClassLoader = Thread.currentThread().getContextClassLoader(); + try { + ProcessorLog log = getLogger(); + + // Need the right classloader when the engine is created. This ensures the NAR's execution class loader + // (plus the module path) becomes the parent for the script engine + ClassLoader scriptEngineModuleClassLoader = createScriptEngineModuleClassLoader(modulePath); + if (scriptEngineModuleClassLoader != null) { + Thread.currentThread().setContextClassLoader(scriptEngineModuleClassLoader); + } + scriptEngine = createScriptEngine(); + ServiceLoader configuratorServiceLoader = + ServiceLoader.load(ScriptEngineConfigurator.class); + for (ScriptEngineConfigurator configurator : configuratorServiceLoader) { + String configuratorScriptEngineName = configurator.getScriptEngineName(); + try { + if (configuratorScriptEngineName != null + && configuratorScriptEngineName.equalsIgnoreCase(scriptEngineName)) { + configurator.init(scriptEngine, modulePath); + scriptEngineConfiguratorMap.put(configurator.getScriptEngineName(), configurator); + } + } catch (ScriptException se) { + log.error("Error initializing script engine configurator {}", + new Object[]{configuratorScriptEngineName}); + if (log.isDebugEnabled()) { + log.error("Error initializing script engine configurator", se); + } + } + } + + } finally { + // Restore original context class loader + Thread.currentThread().setContextClassLoader(originalContextClassLoader); + } + } + + /** + * Provides a ScriptEngine corresponding to the currently selected script engine name. + * ScriptEngineManager.getEngineByName() doesn't use find ScriptEngineFactory.getName(), which + * is what we used to populate the list. So just search the list of factories until a match is + * found, then create and return a script engine. + * + * @return a Script Engine corresponding to the currently specified name, or null if none is found. + */ + protected ScriptEngine createScriptEngine() { + // + ScriptEngineFactory factory = scriptEngineFactoryMap.get(scriptEngineName); + if (factory == null) { + return null; + } + return factory.getScriptEngine(); + } + + /** + * Creates a classloader to be used by the selected script engine and the provided script file. This + * classloader has this class's classloader as a parent (versus the current thread's context + * classloader) and also adds the specified module directory to the classpath. This enables scripts + * to use other scripts, modules, etc. without having to build them into the scripting NAR. + * If the parameter is null or empty, this class's classloader is returned + * + * @param modulePath The path to a directory containing modules to be used by the script(s) + */ + protected ClassLoader createScriptEngineModuleClassLoader(String modulePath) { + URLClassLoader newModuleClassLoader = null; + ClassLoader thisClassLoader = this.getClass().getClassLoader(); + if (StringUtils.isEmpty(modulePath)) { + return thisClassLoader; + } + try { + newModuleClassLoader = + new URLClassLoader( + new URL[]{new File(modulePath).toURI().toURL()}, thisClassLoader); + } catch (MalformedURLException mue) { + getLogger().error("Couldn't find modules directory at " + modulePath, mue); + } + return newModuleClassLoader; + } +} diff --git a/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/ExecuteScript.java b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/ExecuteScript.java new file mode 100644 index 0000000000..36765ea49d --- /dev/null +++ b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/ExecuteScript.java @@ -0,0 +1,213 @@ +/* + * 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.commons.io.IOUtils; +import org.apache.nifi.annotation.behavior.DynamicProperty; +import org.apache.nifi.annotation.behavior.TriggerSerially; +import org.apache.nifi.annotation.lifecycle.OnStopped; +import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.logging.ProcessorLog; +import org.apache.nifi.annotation.lifecycle.OnScheduled; +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.documentation.Tags; +import org.apache.nifi.processor.ProcessContext; +import org.apache.nifi.processor.ProcessSession; +import org.apache.nifi.processor.ProcessSessionFactory; +import org.apache.nifi.processor.Relationship; +import org.apache.nifi.processor.exception.ProcessException; +import org.apache.nifi.processor.util.StandardValidators; + +import javax.script.Bindings; +import javax.script.ScriptContext; +import javax.script.ScriptException; +import javax.script.SimpleBindings; +import java.io.FileInputStream; +import java.io.IOException; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; + +@TriggerSerially +@Tags({"script", "execute", "groovy", "python", "jython", "jruby", "ruby", "javascript", "js", "lua", "luaj"}) +@CapabilityDescription("Experimental - Executes a script given the flow file and a process session. The script is responsible for " + + "handling the incoming flow file (transfer to SUCCESS or remove, e.g.) as well as any flow files created by " + + "the script. If the handling is incomplete or incorrect, the session will be rolled back. Experimental: " + + "Impact of sustained usage not yet verified.") +@DynamicProperty( + name = "A script engine property to update", + value = "The value to set it to", + supportsExpressionLanguage = true, + description = "Updates a script engine property specified by the Dynamic Property's key with the value " + + "specified by the Dynamic Property's value") +public class ExecuteScript extends AbstractScriptProcessor { + + private String scriptToRun = null; + + /** + * Returns the valid relationships for this processor. + * + * @return a Set of Relationships supported by this processor + */ + @Override + public Set getRelationships() { + final Set relationships = new HashSet<>(); + relationships.add(REL_SUCCESS); + relationships.add(REL_FAILURE); + return Collections.unmodifiableSet(relationships); + } + + /** + * Returns a list of property descriptors supported by this processor. The list always includes properties such as + * script engine name, script file name, script body name, script arguments, and an external module path. If the + * scripted processor also defines supported properties, those are added to the list as well. + * + * @return a List of PropertyDescriptor objects supported by this processor + */ + @Override + protected List getSupportedPropertyDescriptors() { + synchronized (isInitialized) { + if (!isInitialized.get()) { + createResources(); + } + } + + return Collections.unmodifiableList(descriptors); + } + + /** + * Returns a PropertyDescriptor for the given name. This is for the user to be able to define their own properties + * which will be available as variables in the script + * + * @param propertyDescriptorName used to lookup if any property descriptors exist for that name + * @return a PropertyDescriptor object corresponding to the specified dynamic property name + */ + @Override + protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) { + return new PropertyDescriptor.Builder() + .name(propertyDescriptorName) + .required(false) + .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) + .expressionLanguageSupported(true) + .dynamic(true) + .build(); + } + + + /** + * Performs setup operations when the processor is scheduled to run. This includes evaluating the processor's + * properties, as well as reloading the script (from file or the "Script Body" property) + * + * @param context the context in which to perform the setup operations + */ + @OnScheduled + public void setup(final ProcessContext context) { + scriptEngineName = context.getProperty(SCRIPT_ENGINE).getValue(); + scriptPath = context.getProperty(SCRIPT_FILE).evaluateAttributeExpressions().getValue(); + scriptBody = context.getProperty(SCRIPT_BODY).getValue(); + modulePath = context.getProperty(MODULES).evaluateAttributeExpressions().getValue(); + super.setup(); + scriptToRun = scriptBody; + + try { + if (scriptToRun == null && scriptPath != null) { + scriptToRun = IOUtils.toString(new FileInputStream(scriptPath)); + } + } catch (IOException ioe) { + throw new ProcessException(ioe); + } + + } + + /** + * Evaluates the given script body (or file) using the current session, context, and flowfile. The script + * evaluation expects a FlowFile to be returned, in which case it will route the FlowFile to success. If a script + * error occurs, the original FlowFile will be routed to failure. If the script succeeds but does not return a + * FlowFile, the original FlowFile will be routed to no-flowfile + * + * @param context the current process context + * @param sessionFactory provides access to a {@link ProcessSessionFactory}, which + * can be used for accessing FlowFiles, etc. + * @throws ProcessException if the scripted processor's onTrigger() method throws an exception + */ + @Override + public void onTrigger(ProcessContext context, ProcessSessionFactory sessionFactory) throws ProcessException { + synchronized (isInitialized) { + if (!isInitialized.get()) { + createResources(); + } + } + ProcessorLog log = getLogger(); + ProcessSession session = sessionFactory.createSession(); + try { + + try { + Bindings bindings = scriptEngine.getBindings(ScriptContext.ENGINE_SCOPE); + if (bindings == null) { + bindings = new SimpleBindings(); + } + bindings.put("session", session); + bindings.put("context", context); + bindings.put("log", log); + bindings.put("REL_SUCCESS", REL_SUCCESS); + bindings.put("REL_FAILURE", REL_FAILURE); + + // Find the user-added properties and set them on the script + for (Map.Entry property : context.getProperties().entrySet()) { + if (property.getKey().isDynamic()) { + // Add the dynamic property bound to its full PropertyValue to the script engine + if (property.getValue() != null) { + bindings.put(property.getKey().getName(), context.getProperty(property.getKey())); + } + } + } + + scriptEngine.setBindings(bindings, ScriptContext.ENGINE_SCOPE); + + // Execute any engine-specific configuration before the script is evaluated + ScriptEngineConfigurator configurator = + scriptEngineConfiguratorMap.get(scriptEngineName); + + // Evaluate the script with the configurator (if it exists) or the engine + if (configurator != null) { + configurator.eval(scriptEngine, scriptToRun, modulePath); + } else { + scriptEngine.eval(scriptToRun); + } + + // Commit this session for the user. This plus the outermost catch statement mimics the behavior + // of AbstractProcessor. This class doesn't extend AbstractProcessor in order to share a base + // class with InvokeScriptedProcessor + session.commit(); + } catch (ScriptException e) { + throw new ProcessException(e); + } + } catch (final Throwable t) { + // Mimic AbstractProcessor behavior here + getLogger().error("{} failed to process due to {}; rolling back session", new Object[]{this, t}); + session.rollback(true); + throw t; + } + } + + @OnStopped + public void stop() { + scriptEngine = null; + } +} diff --git a/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/InvokeScriptedProcessor.java b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/InvokeScriptedProcessor.java new file mode 100644 index 0000000000..ed7d23bd4e --- /dev/null +++ b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/InvokeScriptedProcessor.java @@ -0,0 +1,554 @@ +/* + * 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.commons.io.IOUtils; +import org.apache.commons.lang3.StringUtils; +import org.apache.nifi.annotation.behavior.DynamicProperty; +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.documentation.SeeAlso; +import org.apache.nifi.annotation.documentation.Tags; +import org.apache.nifi.annotation.lifecycle.OnScheduled; +import org.apache.nifi.annotation.lifecycle.OnStopped; +import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.components.ValidationContext; +import org.apache.nifi.components.ValidationResult; +import org.apache.nifi.controller.ControllerServiceLookup; +import org.apache.nifi.logging.ProcessorLog; +import org.apache.nifi.processor.ProcessContext; +import org.apache.nifi.processor.ProcessSessionFactory; +import org.apache.nifi.processor.Processor; +import org.apache.nifi.processor.ProcessorInitializationContext; +import org.apache.nifi.processor.Relationship; +import org.apache.nifi.processor.exception.ProcessException; +import org.apache.nifi.processor.util.StandardValidators; +import org.apache.nifi.util.file.monitor.SynchronousFileWatcher; + +import javax.script.Invocable; +import javax.script.ScriptException; +import java.io.FileInputStream; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +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; + +@Tags({"script", "invoke", "groovy", "python", "jython", "jruby", "ruby", "javascript", "js", "lua", "luaj"}) +@CapabilityDescription("Experimental - Invokes a script engine for a Processor defined in the given script. The script must define " + + "a valid class that implements the Processor interface, and it must set a variable 'processor' to an instance of " + + "the class. Processor methods such as onTrigger() will be delegated to the scripted Processor instance. Also any " + + "Relationships or PropertyDescriptors defined by the scripted processor will be added to the configuration dialog. " + + "Experimental: Impact of sustained usage not yet verified.") +@DynamicProperty(name = "A script engine property to update", value = "The value to set it to", supportsExpressionLanguage = true, + description = "Updates a script engine property specified by the Dynamic Property's key with the value specified by the Dynamic Property's value") +@SeeAlso({ExecuteScript.class}) +public class InvokeScriptedProcessor extends AbstractScriptProcessor { + + private final AtomicReference processor = new AtomicReference<>(); + private final AtomicReference> validationResults = + new AtomicReference<>((Collection) new ArrayList()); + + private final Lock lock = new ReentrantLock(); + private SynchronousFileWatcher scriptWatcher; + + private ScheduledExecutorService reloadService = null; + + private AtomicBoolean scriptNeedsReload = new AtomicBoolean(true); + + /** + * Creates the resources needed by this processor. An attempt is made to also initialize the scripted processor, + * but unless the properties (such as script engine name and script file path) have already been specified, the + * script will not yet have been evaluated, so the script's initialize() method will not be called. + */ + protected void createResources() { + + // Set up script file reloader service. This checks to see if the script file has changed, and if so, tries + // to reload it + if (reloadService == null) { + reloadService = Executors.newScheduledThreadPool(1); + + // monitor the script if configured for changes + reloadService.scheduleWithFixedDelay(new Runnable() { + @Override + public void run() { + try { + final boolean hasLock = lock.tryLock(); + + // if a property is changing we don't need to reload this iteration + if (hasLock) { + try { + if (scriptWatcher != null && scriptWatcher.checkAndReset()) { + if (isFile(scriptPath)) { + // reload the actual script + final boolean reloaded = reloadScriptFile(scriptPath); + + // log the script was reloaded + if (reloaded) { + getLogger().info("The configured script has been successfully reloaded."); + } + } + } + } finally { + lock.unlock(); + } + } + } catch (final Throwable t) { + final ProcessorLog logger = getLogger(); + final String message = "Unable to reload configured script Processor: " + t; + + logger.error(message); + if (logger.isDebugEnabled()) { + logger.error(message, t); + } + } + } + }, 30, 10, TimeUnit.SECONDS); + } + + super.createResources(); + } + + /** + * Returns the valid relationships for this processor. SUCCESS and FAILURE are always returned, and if the script + * processor has defined additional relationships, those will be added as well. + * + * @return a Set of Relationships supported by this processor + */ + @Override + public Set getRelationships() { + final Set relationships = new HashSet<>(); + final Processor instance = processor.get(); + if (instance != null) { + try { + relationships.addAll(instance.getRelationships()); + } catch (final Throwable t) { + final ProcessorLog logger = getLogger(); + final String message = "Unable to get relationships from scripted Processor: " + t; + + logger.error(message); + if (logger.isDebugEnabled()) { + logger.error(message, t); + } + } + } else { + // Return defaults for now + relationships.add(REL_SUCCESS); + relationships.add(REL_FAILURE); + } + return Collections.unmodifiableSet(relationships); + } + + /** + * Returns a list of property descriptors supported by this processor. The list always includes properties such as + * script engine name, script file name, script body name, script arguments, and an external module path. If the + * scripted processor also defines supported properties, those are added to the list as well. + * + * @return a List of PropertyDescriptor objects supported by this processor + */ + @Override + protected List getSupportedPropertyDescriptors() { + + synchronized (isInitialized) { + if (!isInitialized.get()) { + createResources(); + } + } + List supportedPropertyDescriptors = new ArrayList<>(); + supportedPropertyDescriptors.addAll(descriptors); + + final Processor instance = processor.get(); + if (instance != null) { + try { + final List instanceDescriptors = instance.getPropertyDescriptors(); + if (instanceDescriptors != null) { + supportedPropertyDescriptors.addAll(instanceDescriptors); + } + } catch (final Throwable t) { + final ProcessorLog logger = getLogger(); + final String message = "Unable to get property descriptors from Processor: " + t; + + logger.error(message); + if (logger.isDebugEnabled()) { + logger.error(message, t); + } + } + } + + return Collections.unmodifiableList(supportedPropertyDescriptors); + } + + /** + * Returns a PropertyDescriptor for the given name. This is for the user to be able to define their own properties + * which will be available as variables in the script + * + * @param propertyDescriptorName used to lookup if any property descriptors exist for that name + * @return a PropertyDescriptor object corresponding to the specified dynamic property name + */ + @Override + protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) { + return new PropertyDescriptor.Builder() + .name(propertyDescriptorName) + .required(false) + .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) + .expressionLanguageSupported(true) + .dynamic(true) + .build(); + } + + /** + * Performs setup operations when the processor is scheduled to run. This includes evaluating the processor's + * properties, as well as reloading the script (from file or the "Script Body" property) + * + * @param context the context in which to perform the setup operations + */ + @OnScheduled + public void setup(final ProcessContext context) { + scriptEngineName = context.getProperty(SCRIPT_ENGINE).getValue(); + scriptPath = context.getProperty(SCRIPT_FILE).evaluateAttributeExpressions().getValue(); + scriptBody = context.getProperty(SCRIPT_BODY).getValue(); + modulePath = context.getProperty(MODULES).evaluateAttributeExpressions().getValue(); + setup(); + } + + @Override + public void setup() { + super.setup(); + if (scriptNeedsReload.get() || processor.get() == null) { + if (isFile(scriptPath)) { + reloadScriptFile(scriptPath); + } else { + reloadScriptBody(scriptBody); + } + scriptNeedsReload.set(false); + } + } + + + /** + * Handles changes to this processor's properties. If changes are made to script- or engine-related properties, + * the script will be reloaded. + * + * @param descriptor of the modified property + * @param oldValue non-null property value (previous) + * @param newValue the new property value or if null indicates the property + */ + @Override + public void onPropertyModified(final PropertyDescriptor descriptor, final String oldValue, final String newValue) { + final ProcessorLog logger = getLogger(); + final Processor instance = processor.get(); + + if (SCRIPT_FILE.equals(descriptor) + || SCRIPT_BODY.equals(descriptor) + || MODULES.equals(descriptor) + || SCRIPT_ENGINE.equals(descriptor)) { + scriptNeedsReload.set(true); + } else { + if (instance != null) { + // If the script provides a Processor, call its onPropertyModified() method + try { + instance.onPropertyModified(descriptor, oldValue, newValue); + } catch (final Exception e) { + final String message = "Unable to invoke onPropertyModified from script Processor: " + e; + logger.error(message, e); + } + } + } + } + + /** + * Reloads the script located at the given path + * + * @param scriptPath the path to the script file to be loaded + * @return true if the script was loaded successfully; false otherwise + */ + private boolean reloadScriptFile(final String scriptPath) { + final Collection results = new HashSet<>(); + + try (final FileInputStream scriptStream = new FileInputStream(scriptPath)) { + return reloadScript(IOUtils.toString(scriptStream)); + + } catch (final Exception e) { + final ProcessorLog logger = getLogger(); + final String message = "Unable to load script: " + e; + + // If the module path has not yet been set, then this script is likely being loaded too early and depends + // on modules the processor does not yet know about. If this is the case, it will be reloaded later on + // property change (modules) or when scheduled + if (modulePath != null) { + logger.error(message, e); + results.add(new ValidationResult.Builder() + .subject("ScriptValidation") + .valid(false) + .explanation("Unable to load script due to " + e) + .input(scriptPath) + .build()); + } + } + + // store the updated validation results + validationResults.set(results); + + // return whether there was any issues loading the configured script + return results.isEmpty(); + } + + /** + * Reloads the script defined by the given string + * + * @param scriptBody the contents of the script to be loaded + * @return true if the script was loaded successfully; false otherwise + */ + private boolean reloadScriptBody(final String scriptBody) { + final Collection results = new HashSet<>(); + try { + return reloadScript(scriptBody); + + } catch (final Exception e) { + final ProcessorLog logger = getLogger(); + final String message = "Unable to load script: " + e; + + logger.error(message, e); + results.add(new ValidationResult.Builder() + .subject("ScriptValidation") + .valid(false) + .explanation("Unable to load script due to " + e) + .input(scriptPath) + .build()); + } + + // store the updated validation results + validationResults.set(results); + + // return whether there was any issues loading the configured script + return results.isEmpty(); + } + + /** + * Reloads the script Processor. This must be called within the lock. + * + * @param scriptBody An input stream associated with the script content + * @return Whether the script was successfully reloaded + */ + private boolean reloadScript(final String scriptBody) { + // note we are starting here with a fresh listing of validation + // results since we are (re)loading a new/updated script. any + // existing validation results are not relevant + final Collection results = new HashSet<>(); + + try { + // get the engine and ensure its invocable + if (scriptEngine instanceof Invocable) { + final Invocable invocable = (Invocable) scriptEngine; + + // Find a custom configurator and invoke their eval() method + ScriptEngineConfigurator configurator = scriptEngineConfiguratorMap.get(scriptEngineName); + if (configurator != null) { + configurator.eval(scriptEngine, scriptBody, modulePath); + } else { + // evaluate the script + scriptEngine.eval(scriptBody); + } + + // get configured processor from the script (if it exists) + final Object obj = scriptEngine.get("processor"); + if (obj != null) { + final ProcessorLog logger = getLogger(); + + try { + // set the logger if the processor wants it + invocable.invokeMethod(obj, "setLogger", logger); + } catch (final NoSuchMethodException nsme) { + if (logger.isDebugEnabled()) { + logger.debug("Configured script Processor does not contain a setLogger method."); + } + } + + // record the processor for use later + final Processor scriptProcessor = invocable.getInterface(obj, Processor.class); + processor.set(scriptProcessor); + + if (scriptProcessor != null) { + try { + scriptProcessor.initialize(new ProcessorInitializationContext() { + @Override + public String getIdentifier() { + return InvokeScriptedProcessor.this.getIdentifier(); + } + + @Override + public ProcessorLog getLogger() { + return logger; + } + + @Override + public ControllerServiceLookup getControllerServiceLookup() { + return InvokeScriptedProcessor.super.getControllerServiceLookup(); + } + }); + } catch (final Exception e) { + logger.error("Unable to initialize scripted Processor: " + e.getLocalizedMessage(), e); + throw new ProcessException(e); + } + } + } else { + throw new ScriptException("No processor was defined by the script."); + } + } + + } catch (final Exception ex) { + final ProcessorLog logger = getLogger(); + final String message = "Unable to load script: " + ex.getLocalizedMessage(); + + logger.error(message, ex); + results.add(new ValidationResult.Builder() + .subject("ScriptValidation") + .valid(false) + .explanation("Unable to load script due to " + ex.getLocalizedMessage()) + .input(scriptPath) + .build()); + } + + // store the updated validation results + validationResults.set(results); + + // return whether there was any issues loading the configured script + return results.isEmpty(); + } + + /** + * Invokes the validate() routine provided by the script, allowing for custom validation code. + * This method assumes there is a valid Processor defined in the script and it has been loaded + * by the InvokeScriptedProcessor processor + * + * @param context The validation context to be passed into the custom validate method + * @return A collection of ValidationResults returned by the custom validate method + */ + @Override + protected Collection customValidate(final ValidationContext context) { + + // Verify that exactly one of "script file" or "script body" is set + Map propertyMap = context.getProperties(); + if (StringUtils.isEmpty(propertyMap.get(SCRIPT_FILE)) == StringUtils.isEmpty(propertyMap.get(SCRIPT_BODY))) { + Set results = new HashSet<>(); + results.add(new ValidationResult.Builder().valid(false).explanation( + "Exactly one of Script File or Script Body must be set").build()); + return results; + } + + scriptEngineName = context.getProperty(SCRIPT_ENGINE).getValue(); + scriptPath = context.getProperty(SCRIPT_FILE).evaluateAttributeExpressions().getValue(); + scriptBody = context.getProperty(SCRIPT_BODY).getValue(); + modulePath = context.getProperty(MODULES).evaluateAttributeExpressions().getValue(); + setup(); + + // Now that InvokeScriptedProcessor is validated, we can call validate on the scripted processor + final Processor instance = processor.get(); + final Collection currentValidationResults = validationResults.get(); + + // if there was existing validation errors and the processor loaded successfully + if (currentValidationResults.isEmpty() && instance != null) { + try { + // defer to the underlying processor for validation + final Collection instanceResults = instance.validate(context); + if (instanceResults != null && instanceResults.size() > 0) { + // return the validation results from the underlying instance + return instanceResults; + } + } catch (final Exception e) { + final ProcessorLog logger = getLogger(); + final String message = "Unable to validate the script Processor: " + e; + logger.error(message, e); + + // return a new validation message + final Collection results = new HashSet<>(); + results.add(new ValidationResult.Builder() + .subject("Validation") + .valid(false) + .explanation("An error occurred calling validate in the configured script Processor.") + .input(context.getProperty(SCRIPT_FILE).getValue()) + .build()); + return results; + } + } + + return currentValidationResults; + } + + /** + * Invokes the onTrigger() method of the scripted processor. If the script failed to reload, the processor yields + * until the script can be reloaded successfully. If the scripted processor's onTrigger() method throws an + * exception, a ProcessException will be thrown. If no processor is defined by the script, an error is logged + * with the system. + * + * @param context provides access to convenience methods for obtaining + * property values, delaying the scheduling of the processor, provides + * access to Controller Services, etc. + * @param sessionFactory provides access to a {@link ProcessSessionFactory}, which + * can be used for accessing FlowFiles, etc. + * @throws ProcessException if the scripted processor's onTrigger() method throws an exception + */ + @Override + public void onTrigger(ProcessContext context, ProcessSessionFactory sessionFactory) throws ProcessException { + + // Initialize the rest of the processor resources if we have not already done so + synchronized (isInitialized) { + if (!isInitialized.get()) { + createResources(); + } + } + + ProcessorLog log = getLogger(); + + // ensure the processor (if it exists) is loaded + final Processor instance = processor.get(); + + // ensure the processor did not fail to reload at some point + final Collection results = validationResults.get(); + if (!results.isEmpty()) { + log.error(String.format("Unable to run because the Processor is not valid: [%s]", + StringUtils.join(results, ", "))); + context.yield(); + return; + } + if (instance != null) { + try { + // run the processor + instance.onTrigger(context, sessionFactory); + } catch (final ProcessException e) { + final String message = String.format("An error occurred executing the configured Processor [%s]: %s", context.getProperty(SCRIPT_FILE).getValue(), e); + log.error(message); + throw e; + } + } else { + log.error("There is no processor defined by the script"); + } + } + + @OnStopped + public void stop() { + processor.set(null); + } +} diff --git a/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/ScriptEngineConfigurator.java b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/ScriptEngineConfigurator.java new file mode 100644 index 0000000000..5367be045f --- /dev/null +++ b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/ScriptEngineConfigurator.java @@ -0,0 +1,35 @@ +/* + * 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 javax.script.ScriptEngine; +import javax.script.ScriptException; + +/** + * This interface describes callback methods used by the ExecuteScript/InvokeScript processors to perform + * engine-specific tasks at various points in the engine lifecycle. + */ +public interface ScriptEngineConfigurator { + + String getScriptEngineName(); + + Object init(ScriptEngine engine, String modulePath) throws ScriptException; + + Object eval(ScriptEngine engine, String scriptBody, String modulePath) throws ScriptException; + +} diff --git a/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/impl/GroovyScriptEngineConfigurator.java b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/impl/GroovyScriptEngineConfigurator.java new file mode 100644 index 0000000000..27b0fdcc04 --- /dev/null +++ b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/impl/GroovyScriptEngineConfigurator.java @@ -0,0 +1,55 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.processors.script.impl; + +import org.apache.nifi.processors.script.ScriptEngineConfigurator; + +import javax.script.ScriptEngine; +import javax.script.ScriptException; + +public class GroovyScriptEngineConfigurator implements ScriptEngineConfigurator { + + private static final String PRELOADS = + "import org.apache.nifi.components.*\n" + + "import org.apache.nifi.flowfile.FlowFile\n" + + "import org.apache.nifi.processor.*\n" + + "import org.apache.nifi.processor.exception.*\n" + + "import org.apache.nifi.processor.io.*\n" + + "import org.apache.nifi.processor.util.*\n" + + "import org.apache.nifi.processors.script.*\n" + + "import org.apache.nifi.logging.ProcessorLog\n"; + + + private ScriptEngine scriptEngine; + + @Override + public String getScriptEngineName() { + return "Groovy"; + } + + @Override + public Object init(ScriptEngine engine, String modulePath) throws ScriptException { + scriptEngine = engine; + return scriptEngine; + } + + @Override + public Object eval(ScriptEngine engine, String scriptBody, String modulePath) throws ScriptException { + scriptEngine = engine; + return engine.eval(PRELOADS + scriptBody); + } +} diff --git a/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/impl/JythonScriptEngineConfigurator.java b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/impl/JythonScriptEngineConfigurator.java new file mode 100644 index 0000000000..d5c8bba9ec --- /dev/null +++ b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/impl/JythonScriptEngineConfigurator.java @@ -0,0 +1,51 @@ +/* + * 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.impl; + +import org.apache.nifi.processors.script.ScriptEngineConfigurator; + +import javax.script.ScriptEngine; +import javax.script.ScriptException; + +/** + * A helper class to configure the Jython engine with any specific requirements + */ +public class JythonScriptEngineConfigurator implements ScriptEngineConfigurator { + + public String getScriptEngineName() { + return "python"; + } + + @Override + public Object init(ScriptEngine engine, String modulePath) throws ScriptException { + return null; + } + + public Object eval(ScriptEngine engine, String scriptBody, String modulePath) throws ScriptException { + Object returnValue = null; + if (engine != null) { + // Need to import the module path inside the engine, in order to pick up + // other Python/Jython modules + engine.eval("import sys"); + if (modulePath != null) { + engine.eval("sys.path.append('" + modulePath + "')"); + } + returnValue = engine.eval(scriptBody); + } + return returnValue; + } +} diff --git a/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor new file mode 100644 index 0000000000..f0c091cc9d --- /dev/null +++ b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor @@ -0,0 +1,16 @@ +# 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. +org.apache.nifi.processors.script.InvokeScriptedProcessor +org.apache.nifi.processors.script.ExecuteScript diff --git a/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/resources/META-INF/services/org.apache.nifi.processors.script.ScriptEngineConfigurator b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/resources/META-INF/services/org.apache.nifi.processors.script.ScriptEngineConfigurator new file mode 100644 index 0000000000..a7ae5a71b7 --- /dev/null +++ b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/resources/META-INF/services/org.apache.nifi.processors.script.ScriptEngineConfigurator @@ -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. + +org.apache.nifi.processors.script.impl.JythonScriptEngineConfigurator +org.apache.nifi.processors.script.impl.GroovyScriptEngineConfigurator diff --git a/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/java/org/apache/nifi/processors/script/BaseScriptTest.java b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/java/org/apache/nifi/processors/script/BaseScriptTest.java new file mode 100644 index 0000000000..eb83b48101 --- /dev/null +++ b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/java/org/apache/nifi/processors/script/BaseScriptTest.java @@ -0,0 +1,72 @@ +/* + * 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.commons.io.FileUtils; +import org.apache.nifi.util.TestRunner; +import org.apache.nifi.util.TestRunners; +import org.junit.BeforeClass; + +import java.io.File; +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Paths; + +import static org.junit.Assert.assertNotNull; + +/** + * An abstract class with common methods, variables, etc. used by scripting processor unit tests + */ +public abstract class BaseScriptTest { + + public final String TEST_RESOURCE_LOCATION = "target/test/resources/"; + + + protected TestRunner runner; + + /** + * Copies all scripts to the target directory because when they are compiled they can leave unwanted .class files. + * + * @throws Exception Any error encountered while testing + */ + @BeforeClass + public static void setupBeforeClass() throws Exception { + FileUtils.copyDirectory(new File("src/test/resources"), new File("target/test/resources")); + } + + public void setupExecuteScript() throws Exception { + final ExecuteScript executeScript = new ExecuteScript(); + // Need to do something to initialize the properties, like retrieve the list of properties + assertNotNull(executeScript.getSupportedPropertyDescriptors()); + runner = TestRunners.newTestRunner(executeScript); + } + + public void setupInvokeScriptProcessor() throws Exception { + final InvokeScriptedProcessor invokeScriptedProcessor = new InvokeScriptedProcessor(); + // Need to do something to initialize the properties, like retrieve the list of properties + assertNotNull(invokeScriptedProcessor.getSupportedPropertyDescriptors()); + runner = TestRunners.newTestRunner(invokeScriptedProcessor); + } + + public String getFileContentsAsString(String path) { + try { + return new String(Files.readAllBytes(Paths.get(path))); + } catch (IOException ioe) { + return null; + } + } +} diff --git a/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/java/org/apache/nifi/processors/script/TestExecuteGroovy.java b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/java/org/apache/nifi/processors/script/TestExecuteGroovy.java new file mode 100644 index 0000000000..133a461303 --- /dev/null +++ b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/java/org/apache/nifi/processors/script/TestExecuteGroovy.java @@ -0,0 +1,291 @@ +/* + * 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.util.MockFlowFile; +import org.junit.Before; +import org.junit.Test; + +import java.nio.charset.StandardCharsets; +import java.util.HashMap; +import java.util.List; + +import static org.junit.Assert.assertEquals; + + +public class TestExecuteGroovy extends BaseScriptTest { + + public final String TEST_CSV_DATA = "gender,title,first,last\n" + + "female,miss,marlene,shaw\n" + + "male,mr,todd,graham"; + + @Before + public void setup() throws Exception { + super.setupExecuteScript(); + } + + /** + * Tests a script file that has provides the body of an onTrigger() function. + * + * @throws Exception Any error encountered while testing + */ + @Test + public void testReadFlowFileContentAndStoreInFlowFileAttributeWithScriptFile() throws Exception { + runner.setValidateExpressionUsage(false); + runner.setProperty(ExecuteScript.SCRIPT_ENGINE, "Groovy"); + runner.setProperty(ExecuteScript.SCRIPT_FILE, TEST_RESOURCE_LOCATION + "groovy/test_onTrigger.groovy"); + runner.setProperty(ExecuteScript.MODULES, "target/test/resources/groovy"); + + runner.assertValid(); + runner.enqueue("test content".getBytes(StandardCharsets.UTF_8)); + runner.run(); + + runner.assertAllFlowFilesTransferred(ExecuteScript.REL_SUCCESS, 1); + final List result = runner.getFlowFilesForRelationship(ExecuteScript.REL_SUCCESS); + result.get(0).assertAttributeEquals("from-content", "test content"); + } + + /** + * Tests a script file that has provides the body of an onTrigger() function. + * + * @throws Exception Any error encountered while testing + */ + @Test + public void testNoIncomingFlowFile() throws Exception { + runner.setValidateExpressionUsage(false); + runner.setProperty(ExecuteScript.SCRIPT_ENGINE, "Groovy"); + runner.setProperty(ExecuteScript.SCRIPT_FILE, TEST_RESOURCE_LOCATION + "groovy/test_onTrigger.groovy"); + runner.setProperty(ExecuteScript.MODULES, "target/test/resources/groovy"); + + runner.assertValid(); + runner.run(); + + runner.assertTransferCount(ExecuteScript.REL_SUCCESS, 0); + runner.assertTransferCount(ExecuteScript.REL_FAILURE, 0); + } + + /** + * Tests a script file that creates and transfers a new flow file. + * + * @throws Exception Any error encountered while testing + */ + @Test + public void testInvalidConfiguration() throws Exception { + runner.setValidateExpressionUsage(false); + runner.setProperty(ExecuteScript.SCRIPT_ENGINE, "Groovy"); + runner.setProperty(ExecuteScript.SCRIPT_FILE, TEST_RESOURCE_LOCATION); + runner.setProperty(ExecuteScript.SCRIPT_BODY, "body"); + + runner.assertNotValid(); + } + + /** + * Tests a script file that creates and transfers a new flow file. + * + * @throws Exception Any error encountered while testing + */ + @Test + public void testCreateNewFlowFileWithScriptFile() throws Exception { + runner.setValidateExpressionUsage(false); + runner.setProperty(ExecuteScript.SCRIPT_ENGINE, "Groovy"); + runner.setProperty(ExecuteScript.SCRIPT_FILE, TEST_RESOURCE_LOCATION + "groovy/test_onTrigger_newFlowFile.groovy"); + runner.setProperty(ExecuteScript.MODULES, "target/test/resources/groovy"); + + runner.assertValid(); + runner.enqueue(TEST_CSV_DATA.getBytes(StandardCharsets.UTF_8)); + runner.run(); + + // The script removes the original file and transfers only the new one + assertEquals(1, runner.getRemovedCount()); + runner.assertAllFlowFilesTransferred(ExecuteScript.REL_SUCCESS, 1); + final List result = runner.getFlowFilesForRelationship(ExecuteScript.REL_SUCCESS); + result.get(0).assertAttributeEquals("filename", "split_cols.txt"); + } + + /** + * Tests a script file that creates and transfers a new flow file. + * + * @throws Exception Any error encountered while testing + */ + @Test + public void testCreateNewFlowFileWithNoInputFile() throws Exception { + runner.setValidateExpressionUsage(false); + runner.setProperty(ExecuteScript.SCRIPT_ENGINE, "Groovy"); + runner.setProperty(ExecuteScript.SCRIPT_BODY, + getFileContentsAsString(TEST_RESOURCE_LOCATION + "groovy/testCreateNewFlowFileWithNoInputFile.groovy") + ); + + runner.assertValid(); + runner.run(); + + runner.assertAllFlowFilesTransferred(ExecuteScript.REL_SUCCESS, 1); + final List result = runner.getFlowFilesForRelationship(ExecuteScript.REL_SUCCESS); + result.get(0).assertAttributeEquals("filename", "newfile"); + } + + /** + * Tests a script file that creates and transfers a new flow file. + * + * @throws Exception Any error encountered while testing + */ + @Test + public void testDynamicProperties() throws Exception { + runner.setValidateExpressionUsage(true); + runner.setProperty(ExecuteScript.SCRIPT_ENGINE, "Groovy"); + runner.setProperty(ExecuteScript.SCRIPT_FILE, TEST_RESOURCE_LOCATION + "groovy/test_dynamicProperties.groovy"); + runner.setProperty("myProp", "${myAttr}"); + + runner.assertValid(); + runner.enqueue(TEST_CSV_DATA.getBytes(StandardCharsets.UTF_8), + new HashMap(1) {{ + put("myAttr", "testValue"); + }}); + runner.run(); + + runner.assertAllFlowFilesTransferred(ExecuteScript.REL_SUCCESS, 1); + final List result = runner.getFlowFilesForRelationship(ExecuteScript.REL_SUCCESS); + result.get(0).assertAttributeEquals("from-content", "testValue"); + } + + /** + * Tests a script file that changes the content of the incoming flowfile. + * + * @throws Exception Any error encountered while testing + */ + @Test + public void testChangeFlowFileWithScriptFile() throws Exception { + runner.setValidateExpressionUsage(false); + runner.setProperty(ExecuteScript.SCRIPT_ENGINE, "Groovy"); + runner.setProperty(ExecuteScript.SCRIPT_FILE, "target/test/resources/groovy/test_onTrigger_changeContent.groovy"); + runner.setProperty(ExecuteScript.MODULES, "target/test/resources/groovy"); + + runner.assertValid(); + runner.enqueue(TEST_CSV_DATA.getBytes(StandardCharsets.UTF_8)); + runner.run(); + + runner.assertAllFlowFilesTransferred(ExecuteScript.REL_SUCCESS, 1); + final List result = runner.getFlowFilesForRelationship(ExecuteScript.REL_SUCCESS); + MockFlowFile resultFile = result.get(0); + resultFile.assertAttributeEquals("selected.columns", "first,last"); + resultFile.assertContentEquals("Marlene Shaw\nTodd Graham\n"); + } + + + /** + * Tests a script that has provides the body of an onTrigger() function. + * + * @throws Exception Any error encountered while testing + */ + @Test + public void testReadFlowFileContentAndStoreInFlowFileAttributeWithScriptBody() throws Exception { + runner.setValidateExpressionUsage(false); + runner.setProperty(ExecuteScript.SCRIPT_ENGINE, "Groovy"); + runner.setProperty(ExecuteScript.SCRIPT_BODY, getFileContentsAsString( + TEST_RESOURCE_LOCATION + "groovy/testReadFlowFileContentAndStoreInFlowFileAttributeWithScriptBody.groovy") + ); + runner.setProperty(ExecuteScript.MODULES, TEST_RESOURCE_LOCATION + "groovy"); + + runner.assertValid(); + runner.enqueue("test content".getBytes(StandardCharsets.UTF_8)); + runner.run(); + + runner.assertAllFlowFilesTransferred(ExecuteScript.REL_SUCCESS, 1); + final List result = runner.getFlowFilesForRelationship(ExecuteScript.REL_SUCCESS); + result.get(0).assertAttributeEquals("from-content", "test content"); + } + + /** + * Tests a script that has provides the body of an onTrigger() function, where the ExecuteScript processor does + * not specify a modules path + * + * @throws Exception Any error encountered while testing + */ + @Test + public void testReadFlowFileContentAndStoreInFlowFileAttributeWithScriptBodyNoModules() throws Exception { + runner.setValidateExpressionUsage(false); + runner.setProperty(ExecuteScript.SCRIPT_ENGINE, "Groovy"); + runner.setProperty(ExecuteScript.SCRIPT_BODY, getFileContentsAsString( + TEST_RESOURCE_LOCATION + "groovy/testReadFlowFileContentAndStoreInFlowFileAttributeWithScriptBodyNoModules.groovy") + ); + runner.assertValid(); + runner.enqueue("test content".getBytes(StandardCharsets.UTF_8)); + runner.run(); + + runner.assertAllFlowFilesTransferred(ExecuteScript.REL_SUCCESS, 1); + final List result = runner.getFlowFilesForRelationship(ExecuteScript.REL_SUCCESS); + result.get(0).assertAttributeEquals("from-content", "test content"); + } + + /** + * Tests a script that does not transfer or remove the original flow file, thereby causing an error during commit. + * + * @throws Exception Any error encountered while testing. Expecting + */ + @Test(expected = AssertionError.class) + public void testScriptNoTransfer() throws Exception { + runner.setValidateExpressionUsage(false); + runner.setProperty(ExecuteScript.SCRIPT_ENGINE, "Groovy"); + runner.setProperty(ExecuteScript.SCRIPT_BODY, getFileContentsAsString( + TEST_RESOURCE_LOCATION + "groovy/testScriptNoTransfer.groovy") + ); + + runner.assertValid(); + runner.enqueue("test content".getBytes(StandardCharsets.UTF_8)); + runner.run(); + + } + + /** + * Tests a script that uses a dynamic property to set a FlowFile attribute. + * + * @throws Exception Any error encountered while testing + */ + @Test + public void testReadFlowFileContentAndStoreInFlowFileCustomAttribute() throws Exception { + runner.setValidateExpressionUsage(false); + runner.setProperty(ExecuteScript.SCRIPT_ENGINE, "Groovy"); + runner.setProperty(ExecuteScript.SCRIPT_BODY, getFileContentsAsString( + TEST_RESOURCE_LOCATION + "groovy/testReadFlowFileContentAndStoreInFlowFileCustomAttribute.groovy") + ); + runner.setProperty("testprop", "test content"); + + runner.assertValid(); + runner.enqueue("test content".getBytes(StandardCharsets.UTF_8)); + runner.run(); + + runner.assertAllFlowFilesTransferred(ExecuteScript.REL_SUCCESS, 1); + final List result = runner.getFlowFilesForRelationship(ExecuteScript.REL_SUCCESS); + result.get(0).assertAttributeEquals("from-content", "test content"); + } + + /** + * Tests a script that throws an Exception within. The expected result is that the FlowFile will be routed to + * failure + * + * @throws Exception Any error encountered while testing + */ + @Test(expected = AssertionError.class) + public void testScriptException() throws Exception { + runner.setValidateExpressionUsage(false); + runner.setProperty(ExecuteScript.SCRIPT_ENGINE, "Groovy"); + runner.setProperty(ExecuteScript.SCRIPT_BODY, "throw new Exception()"); + + runner.assertValid(); + runner.enqueue("test content".getBytes(StandardCharsets.UTF_8)); + runner.run(); + } +} diff --git a/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/java/org/apache/nifi/processors/script/TestExecuteJRuby.java b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/java/org/apache/nifi/processors/script/TestExecuteJRuby.java new file mode 100644 index 0000000000..90e8f85c8c --- /dev/null +++ b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/java/org/apache/nifi/processors/script/TestExecuteJRuby.java @@ -0,0 +1,59 @@ +/* + * 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.util.MockFlowFile; +import org.apache.nifi.util.TestRunner; +import org.apache.nifi.util.TestRunners; +import org.junit.Before; +import org.junit.Test; + +import java.nio.charset.StandardCharsets; +import java.util.List; + + +public class TestExecuteJRuby extends BaseScriptTest { + + @Before + public void setup() throws Exception { + super.setupExecuteScript(); + } + + + /** + * Tests a script that has provides the body of an onTrigger() function. + * + * @throws Exception Any error encountered while testing + */ + @Test + public void testReadFlowFileContentAndStoreInFlowFileAttribute() throws Exception { + final TestRunner runner = TestRunners.newTestRunner(new ExecuteScript()); + runner.setValidateExpressionUsage(false); + runner.setProperty(ExecuteScript.SCRIPT_ENGINE, "ruby"); + runner.setProperty(ExecuteScript.SCRIPT_FILE, "target/test/resources/jruby/test_onTrigger.rb"); + runner.setProperty(ExecuteScript.MODULES, "target/test/resources/jruby"); + + runner.assertValid(); + runner.enqueue("test content".getBytes(StandardCharsets.UTF_8)); + runner.run(); + + runner.assertAllFlowFilesTransferred("success", 1); + final List result = runner.getFlowFilesForRelationship("success"); + result.get(0).assertAttributeEquals("from-content", "test content"); + } + +} diff --git a/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/java/org/apache/nifi/processors/script/TestExecuteJavascript.java b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/java/org/apache/nifi/processors/script/TestExecuteJavascript.java new file mode 100644 index 0000000000..3f50348f3b --- /dev/null +++ b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/java/org/apache/nifi/processors/script/TestExecuteJavascript.java @@ -0,0 +1,58 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.processors.script; + +import org.apache.nifi.util.MockFlowFile; +import org.apache.nifi.util.TestRunner; +import org.apache.nifi.util.TestRunners; +import org.junit.Before; +import org.junit.Test; + +import java.nio.charset.StandardCharsets; +import java.util.List; + + +public class TestExecuteJavascript extends BaseScriptTest { + + @Before + public void setup() throws Exception { + super.setupExecuteScript(); + } + + /** + * Tests a script that has provides the body of an onTrigger() function. + * + * @throws Exception Any error encountered while testing + */ + @Test + public void testReadFlowFileContentAndStoreInFlowFileAttribute() throws Exception { + final TestRunner runner = TestRunners.newTestRunner(new ExecuteScript()); + runner.setValidateExpressionUsage(false); + runner.setProperty(ExecuteScript.SCRIPT_ENGINE, "ECMAScript"); + runner.setProperty(ExecuteScript.SCRIPT_FILE, "target/test/resources/javascript/test_onTrigger.js"); + runner.setProperty(ExecuteScript.MODULES, "target/test/resources/javascript"); + + runner.assertValid(); + runner.enqueue("test content".getBytes(StandardCharsets.UTF_8)); + runner.run(); + + runner.assertAllFlowFilesTransferred("success", 1); + final List result = runner.getFlowFilesForRelationship("success"); + result.get(0).assertAttributeEquals("from-content", "test content"); + } + +} diff --git a/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/java/org/apache/nifi/processors/script/TestExecuteJython.java b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/java/org/apache/nifi/processors/script/TestExecuteJython.java new file mode 100644 index 0000000000..e1938e3c4c --- /dev/null +++ b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/java/org/apache/nifi/processors/script/TestExecuteJython.java @@ -0,0 +1,77 @@ +/* + * 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.util.MockFlowFile; +import org.junit.Before; +import org.junit.Test; + +import java.nio.charset.StandardCharsets; +import java.util.List; + +/** + * Created by mburgess on 1/25/16. + */ +public class TestExecuteJython extends BaseScriptTest { + + @Before + public void setup() throws Exception { + super.setupExecuteScript(); + } + + /** + * Tests a Jython script that has provides the body of an onTrigger() function. + * + * @throws Exception Any error encountered while testing + */ + @Test + public void testReadFlowFileContentAndStoreInFlowFileAttributeWithScriptBody() throws Exception { + runner.setValidateExpressionUsage(false); + runner.setProperty(ExecuteScript.SCRIPT_ENGINE, "python"); + runner.setProperty(ExecuteScript.SCRIPT_BODY, + "from org.apache.nifi.processors.script import ExecuteScript\n" + + "flowFile = session.get()\n" + + "flowFile = session.putAttribute(flowFile, \"from-content\", \"test content\")\n" + + "session.transfer(flowFile, ExecuteScript.REL_SUCCESS)"); + + runner.assertValid(); + runner.enqueue("test content".getBytes(StandardCharsets.UTF_8)); + runner.run(); + + runner.assertAllFlowFilesTransferred(ExecuteScript.REL_SUCCESS, 1); + final List result = runner.getFlowFilesForRelationship(ExecuteScript.REL_SUCCESS); + result.get(0).assertAttributeEquals("from-content", "test content"); + } + + /** + * Tests a script that does not transfer or remove the original flow file, thereby causing an error during commit. + * + * @throws Exception Any error encountered while testing. Expecting + */ + @Test(expected = AssertionError.class) + public void testScriptNoTransfer() throws Exception { + runner.setValidateExpressionUsage(false); + runner.setProperty(ExecuteScript.SCRIPT_ENGINE, "python"); + runner.setProperty(ExecuteScript.SCRIPT_BODY, + "flowFile = session.putAttribute(flowFile, \"from-content\", \"test content\")\n"); + + runner.assertValid(); + runner.enqueue("test content".getBytes(StandardCharsets.UTF_8)); + runner.run(); + + } +} diff --git a/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/java/org/apache/nifi/processors/script/TestExecuteLua.java b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/java/org/apache/nifi/processors/script/TestExecuteLua.java new file mode 100644 index 0000000000..e0415de0ea --- /dev/null +++ b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/java/org/apache/nifi/processors/script/TestExecuteLua.java @@ -0,0 +1,59 @@ +/* + * 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.util.MockFlowFile; +import org.apache.nifi.util.TestRunner; +import org.apache.nifi.util.TestRunners; +import org.junit.Before; +import org.junit.Test; + +import java.nio.charset.StandardCharsets; +import java.util.List; + + +public class TestExecuteLua extends BaseScriptTest { + + @Before + public void setup() throws Exception { + super.setupExecuteScript(); + } + + + /** + * Tests a script that has provides the body of an onTrigger() function. + * + * @throws Exception Any error encountered while testing + */ + @Test + public void testReadFlowFileContentAndStoreInFlowFileAttribute() throws Exception { + final TestRunner runner = TestRunners.newTestRunner(new ExecuteScript()); + runner.setValidateExpressionUsage(false); + runner.setProperty(ExecuteScript.SCRIPT_ENGINE, "lua"); + runner.setProperty(ExecuteScript.SCRIPT_FILE, "target/test/resources/lua/test_onTrigger.lua"); + runner.setProperty(ExecuteScript.MODULES, "target/test/resources/lua"); + + runner.assertValid(); + runner.enqueue("test content".getBytes(StandardCharsets.UTF_8)); + runner.run(); + + runner.assertAllFlowFilesTransferred(ExecuteScript.REL_SUCCESS, 1); + final List result = runner.getFlowFilesForRelationship(ExecuteScript.REL_SUCCESS); + result.get(0).assertAttributeEquals("from-content", "test content"); + } + +} diff --git a/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/java/org/apache/nifi/processors/script/TestInvokeGroovy.java b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/java/org/apache/nifi/processors/script/TestInvokeGroovy.java new file mode 100644 index 0000000000..349b25d3eb --- /dev/null +++ b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/java/org/apache/nifi/processors/script/TestInvokeGroovy.java @@ -0,0 +1,171 @@ +/* + * 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.components.PropertyDescriptor; +import org.apache.nifi.processor.Relationship; +import org.apache.nifi.util.MockFlowFile; +import org.apache.nifi.util.MockProcessContext; +import org.apache.nifi.util.MockProcessorInitializationContext; +import org.apache.nifi.util.MockValidationContext; +import org.apache.nifi.util.TestRunner; +import org.apache.nifi.util.TestRunners; +import org.junit.Before; +import org.junit.Test; + +import java.nio.charset.StandardCharsets; +import java.util.List; +import java.util.Set; + +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; + +public class TestInvokeGroovy extends BaseScriptTest { + + @Before + public void setup() throws Exception { + super.setupInvokeScriptProcessor(); + } + + /** + * Tests a script that has a Groovy Processor that that reads the first line of text from the flowfiles content and stores the value in an attribute of the outgoing flowfile. + * + * @throws Exception Any error encountered while testing + */ + @Test + public void testReadFlowFileContentAndStoreInFlowFileAttribute() throws Exception { + runner.setValidateExpressionUsage(false); + runner.setProperty(InvokeScriptedProcessor.SCRIPT_ENGINE, "Groovy"); + runner.setProperty(InvokeScriptedProcessor.SCRIPT_FILE, "target/test/resources/groovy/test_reader.groovy"); + runner.setProperty(InvokeScriptedProcessor.MODULES, "target/test/resources/groovy"); + + runner.assertValid(); + runner.enqueue("test content".getBytes(StandardCharsets.UTF_8)); + runner.run(); + + runner.assertAllFlowFilesTransferred("success", 1); + final List result = runner.getFlowFilesForRelationship("success"); + result.get(0).assertAttributeEquals("from-content", "test content"); + } + + /** + * Tests a script that has a Groovy Processor that that reads the first line of text from the flowfiles content and + * stores the value in an attribute of the outgoing flowfile. + * + * @throws Exception Any error encountered while testing + */ + @Test + public void testScriptDefinedAttribute() throws Exception { + InvokeScriptedProcessor processor = new InvokeScriptedProcessor(); + MockProcessContext context = new MockProcessContext(processor); + MockProcessorInitializationContext initContext = new MockProcessorInitializationContext(processor, context); + + processor.initialize(initContext); + + context.setProperty(InvokeScriptedProcessor.SCRIPT_ENGINE, "Groovy"); + context.setProperty(InvokeScriptedProcessor.SCRIPT_FILE, "target/test/resources/groovy/test_reader.groovy"); + context.setProperty(InvokeScriptedProcessor.MODULES, "target/test/resources/groovy"); + processor.customValidate(new MockValidationContext(context)); + processor.setup(context); + + List descriptors = processor.getSupportedPropertyDescriptors(); + assertNotNull(descriptors); + assertTrue(descriptors.size() > 0); + boolean found = false; + for (PropertyDescriptor descriptor : descriptors) { + if (descriptor.getName().equals("test-attribute")) { + found = true; + break; + } + } + assertTrue(found); + } + + /** + * Tests a script that has a Groovy Processor that that reads the first line of text from the flowfiles content and + * stores the value in an attribute of the outgoing flowfile. + * + * @throws Exception Any error encountered while testing + */ + @Test + public void testScriptDefinedRelationship() throws Exception { + InvokeScriptedProcessor processor = new InvokeScriptedProcessor(); + MockProcessContext context = new MockProcessContext(processor); + MockProcessorInitializationContext initContext = new MockProcessorInitializationContext(processor, context); + + processor.initialize(initContext); + + context.setProperty(InvokeScriptedProcessor.SCRIPT_ENGINE, "Groovy"); + context.setProperty(InvokeScriptedProcessor.SCRIPT_FILE, "target/test/resources/groovy/test_reader.groovy"); + processor.customValidate(new MockValidationContext(context)); + processor.setup(context); + + Set relationships = processor.getRelationships(); + assertNotNull(relationships); + assertTrue(relationships.size() > 0); + boolean found = false; + for (Relationship relationship : relationships) { + if (relationship.getName().equals("test")) { + found = true; + break; + } + } + assertTrue(found); + } + + /** + * Tests a script that throws a ProcessException within. The expected result is that the exception will be + * propagated + * + * @throws Exception Any error encountered while testing + */ + @Test(expected = AssertionError.class) + public void testInvokeScriptCausesException() throws Exception { + final TestRunner runner = TestRunners.newTestRunner(new InvokeScriptedProcessor()); + runner.setValidateExpressionUsage(false); + runner.setProperty(InvokeScriptedProcessor.SCRIPT_ENGINE, "Groovy"); + runner.setProperty(ExecuteScript.SCRIPT_BODY, getFileContentsAsString( + TEST_RESOURCE_LOCATION + "groovy/testInvokeScriptCausesException.groovy") + ); + runner.assertValid(); + runner.enqueue("test content".getBytes(StandardCharsets.UTF_8)); + runner.run(); + + } + + /** + * Tests a script that routes the FlowFile to failure. + * + * @throws Exception Any error encountered while testing + */ + @Test + public void testScriptRoutesToFailure() throws Exception { + runner.setValidateExpressionUsage(false); + runner.setProperty(InvokeScriptedProcessor.SCRIPT_ENGINE, "Groovy"); + runner.setProperty(ExecuteScript.SCRIPT_BODY, getFileContentsAsString( + TEST_RESOURCE_LOCATION + "groovy/testScriptRoutesToFailure.groovy") + ); + runner.assertValid(); + runner.enqueue("test content".getBytes(StandardCharsets.UTF_8)); + runner.run(); + + runner.assertAllFlowFilesTransferred(InvokeScriptedProcessor.REL_FAILURE, 1); + final List result = runner.getFlowFilesForRelationship(InvokeScriptedProcessor.REL_FAILURE); + assertFalse(result.isEmpty()); + } +} diff --git a/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/java/org/apache/nifi/processors/script/TestInvokeJython.java b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/java/org/apache/nifi/processors/script/TestInvokeJython.java new file mode 100755 index 0000000000..b45c322187 --- /dev/null +++ b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/java/org/apache/nifi/processors/script/TestInvokeJython.java @@ -0,0 +1,169 @@ +/* + * 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.components.ValidationResult; +import org.apache.nifi.util.MockFlowFile; +import org.apache.nifi.util.MockProcessContext; +import org.apache.nifi.util.TestRunner; +import org.apache.nifi.util.TestRunners; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.nio.charset.StandardCharsets; +import java.util.Collection; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +public class TestInvokeJython extends BaseScriptTest { + + /** + * Copies all scripts to the target directory because when they are compiled they can leave unwanted .class files. + * + * @throws Exception Any error encountered while testing + */ + @Before + public void setup() throws Exception { + super.setupInvokeScriptProcessor(); + } + + /** + * Tests a script that has a Jython processor that is always invalid. + * + * @throws Exception Any error encountered while testing + */ + @Test + public void testAlwaysInvalid() throws Exception { + final TestRunner runner = TestRunners.newTestRunner(new InvokeScriptedProcessor()); + runner.setValidateExpressionUsage(false); + runner.setProperty(InvokeScriptedProcessor.SCRIPT_ENGINE, "python"); + runner.setProperty(InvokeScriptedProcessor.SCRIPT_FILE, "target/test/resources/jython/test_invalid.py"); + + final Collection results = ((MockProcessContext) runner.getProcessContext()).validate(); + Assert.assertEquals(1L, results.size()); + Assert.assertEquals("Never valid.", results.iterator().next().getExplanation()); + } + + /** + * Test a script that has a Jython processor that reads a value from a processor property and another from a flowfile attribute then stores both in the attributes of the flowfile being routed. + *

+ * This may seem contrived but it verifies that the Jython processors properties are being considered and are able to be set and validated. It verifies the processor is able to access the property + * values and flowfile attribute values during onTrigger. Lastly, it verifies the processor is able to route the flowfile to a relationship it specified. + * + * @throws Exception Any error encountered while testing + */ + @Test + public void testUpdateAttributeFromProcessorPropertyAndFlowFileAttribute() throws Exception { + final TestRunner runner = TestRunners.newTestRunner(new InvokeScriptedProcessor()); + runner.setValidateExpressionUsage(false); + runner.setProperty(InvokeScriptedProcessor.SCRIPT_ENGINE, "python"); + runner.setProperty(InvokeScriptedProcessor.SCRIPT_FILE, "target/test/resources/jython/test_update_attribute.py"); + runner.setProperty("for-attributes", "value-1"); + + final Map attributes = new HashMap<>(); + attributes.put("for-attributes", "value-2"); + + runner.assertValid(); + runner.enqueue(new byte[0], attributes); + runner.run(); + + runner.assertAllFlowFilesTransferred("success", 1); + final List result = runner.getFlowFilesForRelationship("success"); + + // verify reading a property value + result.get(0).assertAttributeEquals("from-property", "value-1"); + + // verify reading an attribute value + result.get(0).assertAttributeEquals("from-attribute", "value-2"); + } + + /** + * Tests a script that has a Jython Processor that that reads the first line of text from the flowfiles content and stores the value in an attribute of the outgoing flowfile. + * + * @throws Exception Any error encountered while testing + */ + @Test + public void testReadFlowFileContentAndStoreInFlowFileAttribute() throws Exception { + final TestRunner runner = TestRunners.newTestRunner(new InvokeScriptedProcessor()); + runner.setValidateExpressionUsage(false); + runner.setProperty(InvokeScriptedProcessor.SCRIPT_ENGINE, "python"); + runner.setProperty(InvokeScriptedProcessor.SCRIPT_FILE, "target/test/resources/jython/test_reader.py"); + runner.setProperty(InvokeScriptedProcessor.MODULES, "target/test/resources/jython"); + + runner.assertValid(); + runner.enqueue("test content".getBytes(StandardCharsets.UTF_8)); + runner.run(); + + runner.assertAllFlowFilesTransferred("success", 1); + final List result = runner.getFlowFilesForRelationship("success"); + result.get(0).assertAttributeEquals("from-content", "test content"); + } + + /** + * Tests compression and decompression using two different InvokeScriptedProcessor processor instances. A string is compressed and decompressed and compared. + * + * @throws Exception Any error encountered while testing + */ + @Test + public void testCompressor() throws Exception { + final TestRunner one = TestRunners.newTestRunner(new InvokeScriptedProcessor()); + one.setValidateExpressionUsage(false); + one.setProperty(InvokeScriptedProcessor.SCRIPT_ENGINE, "python"); + one.setProperty(InvokeScriptedProcessor.SCRIPT_FILE, "target/test/resources/jython/test_compress.py"); + one.setProperty(InvokeScriptedProcessor.MODULES, "target/test/resources/jython"); + one.setProperty("mode", "compress"); + + one.assertValid(); + one.enqueue("test content".getBytes(StandardCharsets.UTF_8)); + one.run(); + + one.assertAllFlowFilesTransferred("success", 1); + final List oneResult = one.getFlowFilesForRelationship("success"); + + final TestRunner two = TestRunners.newTestRunner(new InvokeScriptedProcessor()); + two.setValidateExpressionUsage(false); + two.setProperty(InvokeScriptedProcessor.SCRIPT_ENGINE, "python"); + two.setProperty(InvokeScriptedProcessor.MODULES, "target/test/resources/jython"); + two.setProperty(InvokeScriptedProcessor.SCRIPT_FILE, "target/test/resources/jython/test_compress.py"); + two.setProperty("mode", "decompress"); + + two.assertValid(); + two.enqueue(oneResult.get(0)); + two.run(); + + two.assertAllFlowFilesTransferred("success", 1); + final List twoResult = two.getFlowFilesForRelationship("success"); + Assert.assertEquals("test content", new String(twoResult.get(0).toByteArray(), StandardCharsets.UTF_8)); + } + + /** + * Tests a script file that creates and transfers a new flow file. + * + * @throws Exception Any error encountered while testing + */ + @Test + public void testInvalidConfiguration() throws Exception { + runner.setValidateExpressionUsage(false); + runner.setProperty(ExecuteScript.SCRIPT_ENGINE, "python"); + runner.setProperty(ExecuteScript.SCRIPT_FILE, TEST_RESOURCE_LOCATION); + runner.setProperty(ExecuteScript.SCRIPT_BODY, "body"); + + runner.assertNotValid(); + } +} diff --git a/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/resources/groovy/testCreateNewFlowFileWithNoInputFile.groovy b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/resources/groovy/testCreateNewFlowFileWithNoInputFile.groovy new file mode 100644 index 0000000000..e5344fa436 --- /dev/null +++ b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/resources/groovy/testCreateNewFlowFileWithNoInputFile.groovy @@ -0,0 +1,20 @@ +/* + * 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. + */ +FlowFile flowFile = session.create() +flowFile = session.putAttribute(flowFile, 'from-content', 'test content') +flowFile = session.putAttribute(flowFile, 'filename', 'newfile') +session.transfer(flowFile, REL_SUCCESS) \ No newline at end of file diff --git a/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/resources/groovy/testInvokeScriptCausesException.groovy b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/resources/groovy/testInvokeScriptCausesException.groovy new file mode 100644 index 0000000000..1b2922c515 --- /dev/null +++ b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/resources/groovy/testInvokeScriptCausesException.groovy @@ -0,0 +1,62 @@ +/* + * 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 testInvokeScriptCausesException implements Processor { + + + def ProcessorLog log + + @Override + void initialize(ProcessorInitializationContext context) { + } + + @Override + Set getRelationships() { + return [] as Set + } + + @Override + void onTrigger(ProcessContext context, ProcessSessionFactory sessionFactory) throws ProcessException { + throw new ProcessException(); + } + + @Override + Collection validate(ValidationContext context) { + return null + } + + @Override + PropertyDescriptor getPropertyDescriptor(String name) { + return null + } + + @Override + void onPropertyModified(PropertyDescriptor descriptor, String oldValue, String newValue) { + + } + + @Override + List getPropertyDescriptors() { + return null + } + + @Override + String getIdentifier() { + return null + } +} + +processor = new testInvokeScriptCausesException() \ No newline at end of file diff --git a/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/resources/groovy/testReadFlowFileContentAndStoreInFlowFileAttributeWithScriptBody.groovy b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/resources/groovy/testReadFlowFileContentAndStoreInFlowFileAttributeWithScriptBody.groovy new file mode 100644 index 0000000000..efc208771a --- /dev/null +++ b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/resources/groovy/testReadFlowFileContentAndStoreInFlowFileAttributeWithScriptBody.groovy @@ -0,0 +1,23 @@ +/* + * 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. + */ +FlowFile flowFile = session.get() + +if (flowFile == null) { + return; +} +flowFile = session.putAttribute(flowFile, 'from-content', 'test content') +session.transfer(flowFile, REL_SUCCESS) \ No newline at end of file diff --git a/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/resources/groovy/testReadFlowFileContentAndStoreInFlowFileAttributeWithScriptBodyNoModules.groovy b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/resources/groovy/testReadFlowFileContentAndStoreInFlowFileAttributeWithScriptBodyNoModules.groovy new file mode 100644 index 0000000000..d9b476d864 --- /dev/null +++ b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/resources/groovy/testReadFlowFileContentAndStoreInFlowFileAttributeWithScriptBodyNoModules.groovy @@ -0,0 +1,20 @@ +/* + * 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. + */ +FlowFile flowFile = session.get() +if (flowFile == null) return +flowFile = session.putAttribute(flowFile, 'from-content', 'test content') +session.transfer(flowFile, REL_SUCCESS) \ No newline at end of file diff --git a/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/resources/groovy/testReadFlowFileContentAndStoreInFlowFileCustomAttribute.groovy b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/resources/groovy/testReadFlowFileContentAndStoreInFlowFileCustomAttribute.groovy new file mode 100644 index 0000000000..87ef7508e4 --- /dev/null +++ b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/resources/groovy/testReadFlowFileContentAndStoreInFlowFileCustomAttribute.groovy @@ -0,0 +1,20 @@ +/* + * 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. + */ +FlowFile flowFile = session.get() +if (flowFile == null) return +flowFile = session.putAttribute(flowFile, 'from-content', "${testprop}") +session.transfer(flowFile, REL_SUCCESS) \ No newline at end of file diff --git a/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/resources/groovy/testScriptNoTransfer.groovy b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/resources/groovy/testScriptNoTransfer.groovy new file mode 100644 index 0000000000..c428e4a180 --- /dev/null +++ b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/resources/groovy/testScriptNoTransfer.groovy @@ -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. + */ +FlowFile flowFile = session.get() +if (flowFile == null) return +flowFile = session.putAttribute(flowFile, "from-content", "test content") \ No newline at end of file diff --git a/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/resources/groovy/testScriptRoutesToFailure.groovy b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/resources/groovy/testScriptRoutesToFailure.groovy new file mode 100644 index 0000000000..47074a27bd --- /dev/null +++ b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/resources/groovy/testScriptRoutesToFailure.groovy @@ -0,0 +1,64 @@ +/* + * 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 testScriptRoutesToFailure implements Processor { + + def ProcessorLog log + + @Override + void initialize(ProcessorInitializationContext context) { + } + + @Override + Set getRelationships() { + return [] as Set + } + + @Override + void onTrigger(ProcessContext context, ProcessSessionFactory sessionFactory) throws ProcessException { + def session = sessionFactory.createSession() + def flowFile = session.get() + if(!flowFile) return + session.transfer(flowFile, InvokeScriptedProcessor.REL_FAILURE) + } + + @Override + Collection validate(ValidationContext context) { + return null + } + + @Override + PropertyDescriptor getPropertyDescriptor(String name) { + return null + } + + @Override + void onPropertyModified(PropertyDescriptor descriptor, String oldValue, String newValue) { + + } + + @Override + List getPropertyDescriptors() { + return null + } + + @Override + String getIdentifier() { + return null + } +} + +processor = new testScriptRoutesToFailure() \ No newline at end of file diff --git a/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/resources/groovy/test_dynamicProperties.groovy b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/resources/groovy/test_dynamicProperties.groovy new file mode 100644 index 0000000000..48f1c2b8e0 --- /dev/null +++ b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/resources/groovy/test_dynamicProperties.groovy @@ -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. + */ +def flowFile = session.get(); +if (flowFile == null) { + return; +} +flowFile = session.putAttribute(flowFile, "from-content", myProp.evaluateAttributeExpressions(flowFile).getValue()) +session.transfer(flowFile, REL_SUCCESS) \ No newline at end of file diff --git a/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/resources/groovy/test_onTrigger.groovy b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/resources/groovy/test_onTrigger.groovy new file mode 100644 index 0000000000..e78307dd94 --- /dev/null +++ b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/resources/groovy/test_onTrigger.groovy @@ -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. + */ +def flowFile = session.get(); +if (flowFile == null) { + context?.yield(); +} +else { + flowFile = session.putAttribute(flowFile, "from-content", "test content") + session.transfer(flowFile, REL_SUCCESS) +} \ No newline at end of file diff --git a/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/resources/groovy/test_onTrigger_changeContent.groovy b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/resources/groovy/test_onTrigger_changeContent.groovy new file mode 100644 index 0000000000..3a2a6bb486 --- /dev/null +++ b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/resources/groovy/test_onTrigger_changeContent.groovy @@ -0,0 +1,41 @@ +/* + * 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 flowFile = session.get(); +if (flowFile == null) { + return; +} +def selectedColumns = '' + +flowFile = session.write(flowFile, + { inputStream, outputStream -> + String line + + final BufferedReader inReader = new BufferedReader(new InputStreamReader(inputStream, 'UTF-8')) + line = inReader.readLine() + String[] header = line?.split(',') + selectedColumns = "${header[2]},${header[3]}" + + while (line = inReader.readLine()) { + String[] cols = line.split(',') + // Select/project cols + outputStream.write("${cols[2].capitalize()} ${cols[3].capitalize()}\n".getBytes('UTF-8')) + } + } as StreamCallback) + +flowFile = session?.putAttribute(flowFile, "selected.columns", selectedColumns) +flowFile = session?.putAttribute(flowFile, "filename", "split_cols.txt") +session.transfer(flowFile, ExecuteScript.REL_SUCCESS) \ No newline at end of file diff --git a/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/resources/groovy/test_onTrigger_newFlowFile.groovy b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/resources/groovy/test_onTrigger_newFlowFile.groovy new file mode 100644 index 0000000000..fcec6a9e72 --- /dev/null +++ b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/resources/groovy/test_onTrigger_newFlowFile.groovy @@ -0,0 +1,46 @@ +/* + * 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 flowFile = session.get(); +if (flowFile == null) { + return; +} +FlowFile newFlowFile + +session.read(flowFile, + { inputStream -> + String line + + final BufferedReader inReader = new BufferedReader(new InputStreamReader(inputStream, 'UTF-8')) + line = inReader.readLine() + String[] header = line?.split(',') + newFlowFile = session?.create(flowFile) + newFlowFile = session?.putAttribute(newFlowFile, "selected.columns", "${header[1]},${header[2]}") + newFlowFile = session?.putAttribute(newFlowFile, "filename", "split_cols.txt") + + newFlowFile = session.write(newFlowFile, + { outStream -> + while (line = inReader.readLine()) { + String[] cols = line.split(',') + // Select/project cols + outStream.write("${cols[3].capitalize()}, ${cols[2].capitalize()}\n".getBytes('UTF-8')) + } + } as OutputStreamCallback) + } as InputStreamCallback) + +// transfer +session.transfer(newFlowFile, ExecuteScript.REL_SUCCESS) +session.remove((FlowFile) flowFile) \ No newline at end of file diff --git a/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/resources/groovy/test_reader.groovy b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/resources/groovy/test_reader.groovy new file mode 100644 index 0000000000..9778f87f42 --- /dev/null +++ b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/resources/groovy/test_reader.groovy @@ -0,0 +1,77 @@ +/* + * 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 GroovyProcessor implements Processor { + + def REL_TEST = new Relationship.Builder() + .name("test") + .description("A test relationship") + .build(); + + def descriptor = new PropertyDescriptor.Builder() + .name("test-attribute").addValidator(StandardValidators.NON_EMPTY_VALIDATOR).build() + + @Override + void initialize(ProcessorInitializationContext context) { + + } + + @Override + Set getRelationships() { + return [REL_TEST] as Set + } + + @Override + void onTrigger(ProcessContext context, ProcessSessionFactory sessionFactory) throws ProcessException { + def session = sessionFactory.createSession() + def flowFile = session.get(); + if (flowFile == null) { + return; + } + flowFile = session.putAttribute(flowFile, "from-content", "test content") + // transfer + session.transfer(flowFile, InvokeScriptedProcessor.REL_SUCCESS) + session.commit() + } + + @Override + Collection validate(ValidationContext context) { + return null + } + + @Override + PropertyDescriptor getPropertyDescriptor(String name) { + return (name?.equals("test-attribute") ? descriptor : null) + } + + @Override + void onPropertyModified(PropertyDescriptor descriptor, String oldValue, String newValue) { + + } + + @Override + List getPropertyDescriptors() { + + return [descriptor] as List + } + + @Override + String getIdentifier() { + return null + } +} + +processor = new GroovyProcessor(); \ No newline at end of file diff --git a/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/resources/javascript/test_onTrigger.js b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/resources/javascript/test_onTrigger.js new file mode 100644 index 0000000000..b0418ca0a5 --- /dev/null +++ b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/resources/javascript/test_onTrigger.js @@ -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. + */ +flowFile = session.get(); +if (flowFile != null) { + flowFile = session.putAttribute(flowFile, "from-content", "test content") + session.transfer(flowFile, REL_SUCCESS) +} \ No newline at end of file diff --git a/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/resources/jruby/test_onTrigger.rb b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/resources/jruby/test_onTrigger.rb new file mode 100644 index 0000000000..d8a2b7b85e --- /dev/null +++ b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/resources/jruby/test_onTrigger.rb @@ -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. + +java_import Java::org.apache.nifi.processors.script.ExecuteScript +flowFile = session.get() +if flowFile.nil? + return +end +flowFile = session.putAttribute(flowFile, "from-content", "test content") +session.transfer(flowFile, ExecuteScript::REL_SUCCESS) \ No newline at end of file diff --git a/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/resources/jython/callbacks/__init__.py b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/resources/jython/callbacks/__init__.py new file mode 100755 index 0000000000..cdfa6e1b92 --- /dev/null +++ b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/resources/jython/callbacks/__init__.py @@ -0,0 +1,23 @@ +#! /usr/bin/python +# +# 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. +# + +from read_first_line import ReadFirstLine +from compress import Compress +from decompress import Decompress \ No newline at end of file diff --git a/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/resources/jython/callbacks/compress.py b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/resources/jython/callbacks/compress.py new file mode 100755 index 0000000000..51409a940a --- /dev/null +++ b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/resources/jython/callbacks/compress.py @@ -0,0 +1,49 @@ +#! /usr/bin/python +# +# 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 sys +import traceback +import bz2 +from jarray import zeros +from org.python.core.util import StringUtil +from org.apache.nifi.processor.io import StreamCallback + +class Compress(StreamCallback) : + __line = None; + + def __init__(self) : + pass + + def process(self, input, output) : + try : + comp = bz2.BZ2Compressor() + buf = zeros(8192, "b") + while True : + bytes_read = input.read(buf) + if(bytes_read == -1) : + break + output.write(comp.compress(StringUtil.fromBytes(buf, 0, bytes_read))) + output.write(comp.flush()) + except : + print "Exception in Compress:" + print '-' * 60 + traceback.print_exc(file=sys.stdout) + print '-' * 60 + raise diff --git a/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/resources/jython/callbacks/decompress.py b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/resources/jython/callbacks/decompress.py new file mode 100755 index 0000000000..a45ef92fe4 --- /dev/null +++ b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/resources/jython/callbacks/decompress.py @@ -0,0 +1,48 @@ +#! /usr/bin/python +# +# 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 sys +import traceback +import bz2 +from jarray import zeros +from org.python.core.util import StringUtil +from org.apache.nifi.processor.io import StreamCallback + +class Decompress(StreamCallback) : + __line = None; + + def __init__(self) : + pass + + def process(self, input, output) : + try : + comp = bz2.BZ2Decompressor() + buf = zeros(8192, "b") + while True : + bytes_read = input.read(buf) + if(bytes_read == -1) : + break + output.write(comp.decompress(StringUtil.fromBytes(buf, 0, bytes_read))) + except : + print "Exception in Decompress:" + print '-' * 60 + traceback.print_exc(file=sys.stdout) + print '-' * 60 + raise diff --git a/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/resources/jython/callbacks/read_first_line.py b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/resources/jython/callbacks/read_first_line.py new file mode 100755 index 0000000000..a66909b982 --- /dev/null +++ b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/resources/jython/callbacks/read_first_line.py @@ -0,0 +1,50 @@ +#! /usr/bin/python +# +# 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 sys +import traceback +from org.apache.nifi.processor.io import InputStreamCallback +from java.io import BufferedReader, InputStreamReader + +class ReadFirstLine(InputStreamCallback) : + __line = None; + + def __init__(self) : + pass + + def getLine(self) : + return self.__line + + def process(self, input) : + try : + reader = InputStreamReader(input) + bufferedReader = BufferedReader(reader) + self.__line = bufferedReader.readLine() + except : + print "Exception in Reader:" + print '-' * 60 + traceback.print_exc(file=sys.stdout) + print '-' * 60 + raise + finally : + if bufferedReader is not None : + bufferedReader.close() + if reader is not None : + reader.close() diff --git a/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/resources/jython/test_compress.py b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/resources/jython/test_compress.py new file mode 100755 index 0000000000..e69b8fa18b --- /dev/null +++ b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/resources/jython/test_compress.py @@ -0,0 +1,76 @@ +#! /usr/bin/python +# +# 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 sys +import traceback +from callbacks import Compress, Decompress +from org.apache.nifi.processor import Processor +from org.apache.nifi.processor import Relationship +from org.apache.nifi.components import PropertyDescriptor + +class CompressFlowFile(Processor) : + __rel_success = Relationship.Builder().description("Success").name("success").build() + + def __init__(self) : + pass + + def initialize(self, context) : + pass + + def getRelationships(self) : + return set([self.__rel_success]) + + def validate(self, context) : + pass + + def getPropertyDescriptors(self) : + descriptor = PropertyDescriptor.Builder().name("mode").allowableValues("compress", "decompress").required(True).build() + return [descriptor] + + def onPropertyModified(self, descriptor, newValue, oldValue) : + pass + + def onTrigger(self, context, sessionFactory) : + session = sessionFactory.createSession() + try : + # ensure there is work to do + flowfile = session.get() + if flowfile is None : + return + + if context.getProperty("mode").getValue() == "compress" : + flowfile = session.write(flowfile, Compress()) + else : + flowfile = session.write(flowfile, Decompress()) + + # transfer + session.transfer(flowfile, self.__rel_success) + session.commit() + except : + print sys.exc_info()[0] + print "Exception in TestReader:" + print '-' * 60 + traceback.print_exc(file=sys.stdout) + print '-' * 60 + + session.rollback(true) + raise + +processor = CompressFlowFile() \ No newline at end of file diff --git a/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/resources/jython/test_invalid.py b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/resources/jython/test_invalid.py new file mode 100755 index 0000000000..6d36c8ddee --- /dev/null +++ b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/resources/jython/test_invalid.py @@ -0,0 +1,48 @@ +#! /usr/bin/python +# +# 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. +# + +from org.apache.nifi.processor import Processor +from org.apache.nifi.components import ValidationResult + +class AlwaysInvalid(Processor) : + + def __init__(self) : + pass + + def initialize(self, context) : + pass + + def getRelationships(self) : + pass + + def validate(self, context) : + error = ValidationResult.Builder().subject("Processor Error").valid(False).explanation("Never valid.").build() + return [error] + + def getPropertyDescriptors(self) : + pass + + def onPropertyModified(self, descriptor, newValue, oldValue) : + pass + + def onTrigger(self, context, sessionFactory) : + pass + +processor = AlwaysInvalid() \ No newline at end of file diff --git a/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/resources/jython/test_reader.py b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/resources/jython/test_reader.py new file mode 100755 index 0000000000..fad6d378e3 --- /dev/null +++ b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/resources/jython/test_reader.py @@ -0,0 +1,74 @@ +#! /usr/bin/python +# +# 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 sys +import traceback +from callbacks import ReadFirstLine +from org.apache.nifi.processor import Processor +from org.apache.nifi.processor import Relationship + +class ReadContentAndStoreAsAttribute(Processor) : + __rel_success = Relationship.Builder().description("Success").name("success").build() + + def __init__(self) : + pass + + def initialize(self, context) : + pass + + def getRelationships(self) : + return set([self.__rel_success]) + + def validate(self, context) : + pass + + def getPropertyDescriptors(self) : + pass + + def onPropertyModified(self, descriptor, newValue, oldValue) : + pass + + def onTrigger(self, context, sessionFactory) : + session = sessionFactory.createSession() + try : + # ensure there is work to do + flowfile = session.get() + if flowfile is None : + return + + reader = ReadFirstLine() + session.read(flowfile, reader); + + # set an attribute + flowfile = session.putAttribute(flowfile, "from-content", reader.getLine()) + # transfer + session.transfer(flowfile, self.__rel_success) + session.commit() + except : + print sys.exc_info()[0] + print "Exception in TestReader:" + print '-' * 60 + traceback.print_exc(file=sys.stdout) + print '-' * 60 + + session.rollback(True) + raise + +processor = ReadContentAndStoreAsAttribute() \ No newline at end of file diff --git a/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/resources/jython/test_update_attribute.py b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/resources/jython/test_update_attribute.py new file mode 100755 index 0000000000..a333f7aa21 --- /dev/null +++ b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/resources/jython/test_update_attribute.py @@ -0,0 +1,79 @@ +#! /usr/bin/python +# +# 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 sys +import traceback +from org.apache.nifi.processor import Processor +from org.apache.nifi.processor import Relationship +from org.apache.nifi.components import PropertyDescriptor +from org.apache.nifi.processor.util import StandardValidators + +class UpdateAttributes(Processor) : + __rel_success = Relationship.Builder().description("Success").name("success").build() + + def __init__(self) : + pass + + def initialize(self, context) : + pass + + def getRelationships(self) : + return set([self.__rel_success]) + + def validate(self, context) : + pass + + def getPropertyDescriptors(self) : + descriptor = PropertyDescriptor.Builder().name("for-attributes").addValidator(StandardValidators.NON_EMPTY_VALIDATOR).build() + return [descriptor] + + def onPropertyModified(self, descriptor, newValue, oldValue) : + pass + + def onTrigger(self, context, sessionFactory) : + session = sessionFactory.createSession() + try : + # ensure there is work to do + flowfile = session.get() + if flowfile is None : + return + + # extract some attribute values + fromPropertyValue = context.getProperty("for-attributes").getValue() + fromAttributeValue = flowfile.getAttribute("for-attributes") + + # set an attribute + flowfile = session.putAttribute(flowfile, "from-property", fromPropertyValue) + flowfile = session.putAttribute(flowfile, "from-attribute", fromAttributeValue) + + # transfer + session.transfer(flowfile, self.__rel_success) + session.commit() + except : + print sys.exc_info()[0] + print "Exception in TestReader:" + print '-' * 60 + traceback.print_exc(file=sys.stdout) + print '-' * 60 + + session.rollback(true) + raise + +processor = UpdateAttributes() \ No newline at end of file diff --git a/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/resources/lua/test_onTrigger.lua b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/resources/lua/test_onTrigger.lua new file mode 100644 index 0000000000..687cae15d5 --- /dev/null +++ b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/resources/lua/test_onTrigger.lua @@ -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. +flowFile = session:get() +if flowFile == nil then + return +end +flowFile = session:putAttribute(flowFile, "from-content", "test content") +session:transfer(flowFile, REL_SUCCESS) \ No newline at end of file diff --git a/nifi-nar-bundles/nifi-scripting-bundle/pom.xml b/nifi-nar-bundles/nifi-scripting-bundle/pom.xml new file mode 100644 index 0000000000..f6763fd1df --- /dev/null +++ b/nifi-nar-bundles/nifi-scripting-bundle/pom.xml @@ -0,0 +1,43 @@ + + + + 4.0.0 + + + org.apache.nifi + nifi-nar-bundles + 0.4.2-SNAPSHOT + + + nifi-scripting-bundle + pom + + + nifi-scripting-processors + nifi-scripting-nar + + + + + + org.apache.nifi + nifi-scripting-processors + 0.4.2-SNAPSHOT + + + + + diff --git a/nifi-nar-bundles/pom.xml b/nifi-nar-bundles/pom.xml index 3bc915b7f5..646ae1c49a 100644 --- a/nifi-nar-bundles/pom.xml +++ b/nifi-nar-bundles/pom.xml @@ -51,6 +51,7 @@ nifi-ldap-iaa-providers-bundle nifi-riemann-bundle nifi-html-bundle + nifi-scripting-bundle diff --git a/pom.xml b/pom.xml index b00890bf16..563ecc3f92 100644 --- a/pom.xml +++ b/pom.xml @@ -583,9 +583,9 @@ language governing permissions and limitations under the License. --> org.apache.activemq - activemq-broker + activemq-broker 5.12.1 - tests + test org.apache.lucene @@ -1026,6 +1026,12 @@ language governing permissions and limitations under the License. --> 0.4.2-SNAPSHOT nar + + org.apache.nifi + nifi-scripting-nar + 0.4.2-SNAPSHOT + nar + org.apache.nifi nifi-properties @@ -1409,8 +1415,8 @@ language governing permissions and limitations under the License. --> + are those tests interfacing with external sources and services requiring additional resources or credentials that cannot + be explicitly provided. --> integration-tests