diff --git a/nifi-fn/.dockerignore b/nifi-fn/.dockerignore deleted file mode 100644 index 940200949d..0000000000 --- a/nifi-fn/.dockerignore +++ /dev/null @@ -1,15 +0,0 @@ -# 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. - -.git/ -.idea/ \ No newline at end of file diff --git a/nifi-fn/README.md b/nifi-fn/README.md index 5cfbb1d4a8..9a24f5e7b5 100644 --- a/nifi-fn/README.md +++ b/nifi-fn/README.md @@ -15,15 +15,19 @@ # NiFi-Fn ### Build: -```mvn package``` +`mvn package` -docker image will be tagged nifi-fn:1.9.0-SNAPSHOT +Docker image will be tagged nifi-fn:1.10.0-SNAPSHOT ### Usage: +After building, the Program can be run from the `target` directory: +`java -cp "lib/*" org.apache.nifi.fn.NiFiFn ` + +Where the arguments dictate the runtime to use: ``` 1) RunFromRegistry [Once|Continuous] [] [] RunFromRegistry [Once|Continuous] --json - RunFromRegistry [Once|Continuous] --file + RunFromRegistry [Once|Continuous] --file # Filename of JSON file that matches the examples below. 2) RunYARNServiceFromRegistry <# of Containers> \ [] [] @@ -35,12 +39,15 @@ docker image will be tagged nifi-fn:1.9.0-SNAPSHOT ### Examples: ``` -1) RunFromRegistry Once http://172.0.0.1:61080 e53b8a0d-5c85-4fcd-912a-1c549a586c83 6cf8277a-c402-4957-8623-0fa9890dd45d \ - "DestinationDirectory-/tmp/nififn/output2/" "" "absolute.path-/tmp/nififn/input/;filename-test.txt" "absolute.path-/tmp/nififn/input/;filename-test2.txt" -2) RunFromRegistry Once http://172.0.0.1:61080 e53b8a0d-5c85-4fcd-912a-1c549a586c83 6cf8277a-c402-4957-8623-0fa9890dd45d \ - "DestinationDirectory-/tmp/nififn/output2/" "f25c9204-6c95-3aa9-b0a8-c556f5f61849" "absolute.path-/tmp/nififn/input/;filename-test.txt" -3) RunYARNServiceFromRegistry http://127.0.0.1:8088 nifi-fn:latest kafka-to-solr 3 --file kafka-to-solr.json -4) RunOpenwhiskActionServer 8080 +1) java -cp "lib/*" org.apache.nifi.fn.NiFiFn lib/ work/ \ + RunFromRegistry Once http://172.0.0.1:61080 e53b8a0d-5c85-4fcd-912a-1c549a586c83 6cf8277a-c402-4957-8623-0fa9890dd45d \ + "DestinationDirectory-/tmp/nififn/output2/" "" "absolute.path-/tmp/nififn/input/;filename-test.txt" "absolute.path-/tmp/nififn/input/;filename-test2.txt" +2) java -cp "lib/*" org.apache.nifi.fn.NiFiFn lib/ work/ \ + RunFromRegistry Once --file /Users/nifi/nifi-fn-configs/flow-abc.json +3) java -cp "lib/*" org.apache.nifi.fn.NiFiFn lib/ work/ \ + RunYARNServiceFromRegistry http://127.0.0.1:8088 nifi-fn:latest kafka-to-solr 3 --file kafka-to-solr.json +4) java -cp "lib/*" org.apache.nifi.fn.NiFiFn lib/ work/ \ + RunOpenwhiskActionServer 8080 ``` ###Notes: @@ -54,42 +61,73 @@ docker image will be tagged nifi-fn:1.9.0-SNAPSHOT All other attributes will be passed to the flow using the variable registry interface ``` -###JSON Sample: -``` -{ - "nifi_registry": "http://localhost:61080", - "nifi_bucket": "3aa885db-30c8-4c87-989c-d32b8ea1d3d8", - "nifi_flow": "0d219eb8-419b-42ba-a5ee-ce07445c6fc5", - "nifi_flowversion": -1, - "nifi_materializecontent":true, - "nifi_failureports": ["f25c9204-6c95-3aa9-b0a8-c556f5f61849"], - "nifi_flowfiles":[{ - "absolute.path":"/tmp/nififn/input/", - "filename":"test.txt", +### JSON Format +The JSON that is provided, either via the `--json` command-line argument or the `--file` command-line argument has the following elements: - "nifi_content":"hello" - }, - { - "absolute.path":"/tmp/nififn/input/", - "filename":"test2.txt", +- `registryUrl` : The URL of the NiFi Registry that should be used for pulling the Flow +- `bucketId` : The UUID of the Bucket containing the flow +- `flowId` : The UUID of the flow to run +- `flowVersion` : _Optional_ - The Version of the flow to run. If not present or equal to -1, then the latest version of the flow will be used. +- `materializeContent` : _Optional_ - Whether or not the contents of the FlowFile should be stored in Java Heap so that they can be read multiple times. If this value is `false`, the contents of any +input FlowFile will be read as a stream of data and not buffered into heap. However, this means that the contents can be read only one time. This can be useful if transferring large files from HDFS to + another HDFS instance or directory, for example, and contains a simple flow such as `ListHDFS -> FetchHDFS -> PutHDFS`. In this flow, the contents of the files will be buffered into Java Heap if the + value of this argument is `true` but will not be if the value of this argument is `false`. +- `failurePortIds`: _Optional_ - An array of Port UUID's, such that if any data is sent to one of the ports with these ID's, the flow is considered "failed" and will stop immediately. +- `ssl`: _Optional_ - If present, provides SSL keystore and truststore information that can be used for interacting with the NiFi Registry and for Site-to-Site communications for Remote Process +Groups. +- `flowFiles`: _Optional_ - An array of FlowFiles that should be provided to the flow's Input Port. Each element in the array is a JSON object. That JSON object can have multiple keys. If any of those +keys is `nifi_content` then the String value of that element will be the FlowFile's content. Otherwise, the key/value pair is considered an attribute of the FlowFile. +- `variables`: _Optional_ - Key/value pairs that will be passed to the NiFi Flow as variables of the root Process Group. + + +### Minimal JSON Sample: + { + "registryUrl": "http://localhost:18080", + "bucketId": "3aa885db-30c8-4c87-989c-d32b8ea1d3d8", + "flowId": "0d219eb8-419b-42ba-a5ee-ce07445c6fc5" + } + + +### Full JSON Sample: + { + "registryUrl": "https://localhost:9443", + "bucketId": "3aa885db-30c8-4c87-989c-d32b8ea1d3d8", + "flowId": "0d219eb8-419b-42ba-a5ee-ce07445c6fc5", + "flowVersion": 8, + "materializeContent":true, + "failurePortIds": ["f25c9204-6c95-3aa9-b0a8-c556f5f61849"], + "ssl": { + "keystoreFile": "/etc/security/keystore.jks", + "keystorePass": "apachenifi", + "keyPass": "nifiapache", + "keystoreType": "JKS", + "truststoreFile": "/etc/security/truststore.jks", + "truststorePass": "apachenifi", + "truststoreType": "JKS" + }, + "flowFiles":[{ + "absolute.path": "/tmp/nififn/input/", + "filename": "test.txt", + + "nifi_content": "hello" + }, + { + "absolute.path": "/tmp/nififn/input/", + "filename": "test2.txt", + + "nifi_content": "hi" + }], + "variables": { + "DestinationDirectory" : "/tmp/nififn/output2/" + } + } - "nifi_content":"hi" - }], - "DestinationDirectory":"/tmp/nififn/output2/" -} -``` ### TODO: * Provenance is always recorded instead of waiting for commit. Rollback could result in duplicates: -FnProvenanceReporter.send force option is not appreciated - -NiFi-FnProcessSession.adjustCounter immediate is not appreciated -* Nar directory is hardcoded - reflectionUtil uses /usr/share/nifi-1.8.0/lib/ (location inside dockerfile) -* ####Classloader does not work -* Add support for: - process groups - funnels + -FnProcessSession.adjustCounter immediate is not appreciated * Send logs, metrics, and provenance to kafka/solr (configure a flow ID for each?) * counters * tests diff --git a/nifi-fn/nifi-fn-assembly/LICENSE b/nifi-fn/nifi-fn-assembly/LICENSE new file mode 100644 index 0000000000..27349539cf --- /dev/null +++ b/nifi-fn/nifi-fn-assembly/LICENSE @@ -0,0 +1,313 @@ + + 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 'Hamcrest' which is available +under a BSD license. More details found here: http://hamcrest.org. + + Copyright (c) 2000-2006, www.hamcrest.org + All rights reserved. + + Redistribution and use in source and binary forms, with or without + modification, are permitted provided that the following conditions are met: + + Redistributions of source code must retain the above copyright notice, this list of + conditions and the following disclaimer. Redistributions in binary form must reproduce + the above copyright notice, this list of conditions and the following disclaimer in + the documentation and/or other materials provided with the distribution. + + Neither the name of Hamcrest nor the names of its contributors may be used to endorse + or promote products derived from this software without specific prior written + permission. + + THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND ANY + EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES + OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT + SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, + INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED + TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR + BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN + CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY + WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH + DAMAGE. + +The binary distribution of this product bundles 'Antlr 3' which is available +under a "3-clause BSD" license. For details see http://www.antlr3.org/license.html + + Copyright (c) 2010 Terence Parr + All rights reserved. + + Redistribution and use in source and binary forms, with or without + modification, are permitted provided that the following conditions are met: + + Redistributions of source code must retain the above copyright notice, this + list of conditions and the following disclaimer. + Redistributions in binary form must reproduce the above copyright notice, + this list of conditions and the following disclaimer in the documentation + and/or other materials provided with the distribution. + Neither the name of the author nor the names of its contributors may be used + to endorse or promote products derived from this software without specific + prior written permission. + THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" + AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE + IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE + ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE + LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR + CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF + SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS + INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN + CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) + ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF + THE POSSIBILITY OF SUCH DAMAGE. + + +The binary distribution of this product bundles 'Bouncy Castle JDK 1.5' +under an MIT style license. + + Copyright (c) 2000 - 2015 The Legion of the Bouncy Castle Inc. (http://www.bouncycastle.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. + + +This product bundles 'jBCrypt' which is available under an MIT license. +For details see https://github.com/svenkubiak/jBCrypt/blob/0.4.1/LICENSE + + Copyright (c) 2006 Damien Miller + + Permission to use, copy, modify, and distribute this software for any + purpose with or without fee is hereby granted, provided that the above + copyright notice and this permission notice appear in all copies. + + THE SOFTWARE IS PROVIDED "AS IS" AND THE AUTHOR DISCLAIMS ALL WARRANTIES + WITH REGARD TO THIS SOFTWARE INCLUDING ALL IMPLIED WARRANTIES OF + MERCHANTABILITY AND FITNESS. IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR + ANY SPECIAL, DIRECT, INDIRECT, OR CONSEQUENTIAL DAMAGES OR ANY DAMAGES + WHATSOEVER RESULTING FROM LOSS OF USE, DATA OR PROFITS, WHETHER IN AN + ACTION OF CONTRACT, NEGLIGENCE OR OTHER TORTIOUS ACTION, ARISING OUT OF + OR IN CONNECTION WITH THE USE OR PERFORMANCE OF THIS SOFTWARE. + + + + diff --git a/nifi-fn/nifi-fn-assembly/NOTICE b/nifi-fn/nifi-fn-assembly/NOTICE new file mode 100644 index 0000000000..8168140c7f --- /dev/null +++ b/nifi-fn/nifi-fn-assembly/NOTICE @@ -0,0 +1,144 @@ +Apache NiFi NiFi-Fn +Copyright 2015-2019 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) Jetty + The following NOTICE information applies: + Jetty Web Container + Copyright 1995-2017 Mort Bay Consulting Pty Ltd. + + (ASLv2) Google GSON + The following NOTICE information applies: + Copyright 2008 Google Inc. + + (ASLv2) Apache Commons Text + The following NOTICE information applies: + Apache Commons Text + Copyright 2001-2018 The Apache Software Foundation + + (ASLv2) Jackson JSON processor + The following NOTICE information applies: + # Jackson JSON processor + + Jackson is a high-performance, Free/Open Source JSON processing library. + It was originally written by Tatu Saloranta (tatu.saloranta@iki.fi), and has + been in development since 2007. + It is currently developed by a community of developers, as well as supported + commercially by FasterXML.com. + + ## Licensing + + Jackson core and extension components may licensed under different licenses. + To find the details that apply to this artifact see the accompanying LICENSE file. + For more information, including possible other licensing options, contact + FasterXML.com (http://fasterxml.com). + + ## Credits + + A list of contributors may be found from CREDITS file, which is included + in some artifacts (usually source distributions); but is always available + from the source code management (SCM) system project uses. + + (ASLv2) Apache Commons Codec + The following NOTICE information applies: + Apache Commons Codec + Copyright 2002-2014 The Apache Software Foundation + + src/test/org/apache/commons/codec/language/DoubleMetaphoneTest.java + contains test data from http://aspell.net/test/orig/batch0.tab. + Copyright (C) 2002 Kevin Atkinson (kevina@gnu.org) + + =============================================================================== + + The content of package org.apache.commons.codec.language.bm has been translated + from the original php source code available at http://stevemorse.org/phoneticinfo.htm + with permission from the original authors. + Original source copyright: + Copyright (c) 2008 Alexander Beider & Stephen P. Morse. + + (ASLv2) Apache Commons Lang + The following NOTICE information applies: + Apache Commons Lang + Copyright 2001-2017 The Apache Software Foundation + + This product includes software from the Spring Framework, + under the Apache License 2.0 (see: StringUtils.containsWhitespace()) + + (ASLv2) Quartz + The following NOTICE information applies: + Copyright Declaration: + Copyright © 2003-2016 Software AG, Darmstadt, Germany and/or Software AG USA Inc., Reston, VA, USA, and/or its subsidiaries and/or its affiliates and/or their licensors. + + Trademark and Patent declaration + The name Software AG and all Software AG product names are either trademarks or registered trademarks of Software AG and/or Software AG USA Inc. and/or its subsidiaries and/or its affiliates + and/or their licensors. Other company and product names mentioned herein may be trademarks of their respective owners. + + Detailed information on trademarks and patents owned by Software AG and/or its subsidiaries is located at http://softwareag.com/licenses. + + Third Party declaration + This software may include portions of third-party products. For third-party copyright notices, license terms, additional rights or restrictions, please refer to "License Texts, Copyright + Notices and Disclaimers of Third Party Products". For certain specific third-party license restrictions, please refer to section E of the Legal Notices available under "License Terms and + Conditions for Use of Software AG Products / Copyright and Trademark Notices of Software AG Products". These documents are part of the product documentation, located at + http://softwareag.com/licenses and/or in the root installation directory of the licensed product(s). + + Confidentiality Disclaimer: + Use, reproduction, transfer, publication or disclosure is prohibited except as specifically provided for in your License Agreement with Software AG. + Contact GitHub API Training Shop Blog About + + (ASLv2) Spring Framework + The following NOTICE information applies: + Spring Framework 4.x,5.x.RELEASE + Copyright (c) 2002-2015 Pivotal, Inc. + + (ASLv2) Spring Security + The following NOTICE information applies: + Spring Framework 4.0.3.RELEASE + Copyright (c) 2002-2015 Pivotal, Inc. + + (ASLv2) Ehcache 2.x + The following NOTICE information applies: + Copyright 2003-2010 Terracotta, Inc. + + + + + +************************ +Common Development and Distribution License 1.1 +************************ + +The following binary components are provided under the Common Development and Distribution License 1.1. See project link for details. + + (CDDL 1.1) (GPL2 w/ CPE) Java Servlet API (javax.servlet:javax.servlet-api:jar:3.1.0 - http://servlet-spec.java.net) + (CDDL 1.1) (GPL2 w/ CPE) javax.ws.rs-api (javax.ws.rs:javax.ws.rs-api:jar:2.1 - http://jax-rs-spec.java.net) + (CDDL 1.1) (GPL2 w/ CPE) jersey-client (org.glassfish.jersey.core:jersey-client:jar:2.26 - https://jersey.github.io/) + (CDDL 1.1) (GPL2 w/ CPE) jersey-common (org.glassfish.jersey.core:jersey-common:jar:2.26 - https://jersey.github.io/) + (CDDL 1.1) (GPL2 w/ CPE) jersey-hk2 (org.glassfish.jersey.inject:jersey-hk2:jar:2.26 - https://jersey.github.io/) + (CDDL 1.1) (GPL2 w/ CPE) jersey-media-json-jackson (org.glassfish.jersey.media:jersey-media-json-jackson:jar:2.26 - https://jersey.github.io/) + + +************************ +Eclipse Public License 1.0 +************************ + +The following binary components are provided under the Eclipse Public License 1.0. See project link for details. + + (EPL 1.0)(LGPL 2.1) Logback Classic (ch.qos.logback:logback-classic:jar:1.2.3 - http://logback.qos.ch/) + (EPL 1.0) AspectJ Weaver (org.aspectj:aspectjweaver:jar:1.8.5 - http://www.eclipse.org/aspectj/) + (EPL 1.0) AspectJ Runtime (org.aspectj:aspectjrt:jar:1.8.0 - http://www.eclipse.org/aspectj/) + +***************** +Public Domain +***************** + +The following binary components are provided to the 'Public Domain'. See project link for details. + + (Public Domain) AOP Alliance 1.0 (http://aopalliance.sourceforge.net/) diff --git a/nifi-fn/nifi-fn-assembly/pom.xml b/nifi-fn/nifi-fn-assembly/pom.xml new file mode 100644 index 0000000000..728bdd60b9 --- /dev/null +++ b/nifi-fn/nifi-fn-assembly/pom.xml @@ -0,0 +1,139 @@ + + + + 4.0.0 + + + org.apache.nifi + nifi-fn + 1.10.0-SNAPSHOT + + + nifi-fn-assembly + pom + + + + + maven-assembly-plugin + + nifi-fn-${project.version} + false + + + + make shared resource + + single + + package + + + 0775 + 0775 + 0664 + + + src/main/assembly/dependencies.xml + + posix + + dir + zip + + + + + + + + + + + + org.apache.nifi + nifi-fn-bootstrap + 1.10.0-SNAPSHOT + + + org.apache.nifi + nifi-api + 1.10.0-SNAPSHOT + + + org.apache.nifi + nifi-framework-api + 1.10.0-SNAPSHOT + + + org.apache.nifi + nifi-properties + 1.10.0-SNAPSHOT + + + javax.servlet + javax.servlet-api + 3.1.0 + runtime + + + org.slf4j + slf4j-api + 1.7.25 + runtime + + + ch.qos.logback + logback-classic + compile + + + + + org.apache.nifi + nifi-fn-nar + 1.10.0-SNAPSHOT + nar + + + org.apache.nifi + nifi-standard-services-api-nar + 1.10.0-SNAPSHOT + nar + + + org.apache.nifi + nifi-standard-nar + 1.10.0-SNAPSHOT + nar + + + org.apache.nifi + nifi-update-attribute-nar + 1.10.0-SNAPSHOT + nar + + + org.apache.nifi + nifi-jetty-bundle + 1.10.0-SNAPSHOT + nar + + + + \ No newline at end of file diff --git a/nifi-fn/nifi-fn-assembly/src/main/assembly/dependencies.xml b/nifi-fn/nifi-fn-assembly/src/main/assembly/dependencies.xml new file mode 100644 index 0000000000..f4bfc5df0d --- /dev/null +++ b/nifi-fn/nifi-fn-assembly/src/main/assembly/dependencies.xml @@ -0,0 +1,32 @@ + + + + bin + true + nifi-fn-${project.version} + + + + + false + lib + 0770 + 0664 + false + + + + diff --git a/nifi-fn/nifi-fn-bootstrap/pom.xml b/nifi-fn/nifi-fn-bootstrap/pom.xml new file mode 100644 index 0000000000..c672c8da49 --- /dev/null +++ b/nifi-fn/nifi-fn-bootstrap/pom.xml @@ -0,0 +1,40 @@ + + + + 4.0.0 + + + org.apache.nifi + nifi-fn + 1.10.0-SNAPSHOT + + + nifi-fn-bootstrap + + + + org.apache.nifi + nifi-nar-utils + 1.10.0-SNAPSHOT + + + org.slf4j + slf4j-api + + + \ No newline at end of file diff --git a/nifi-fn/nifi-fn-bootstrap/src/main/java/org/apache/nifi/fn/NiFiFn.java b/nifi-fn/nifi-fn-bootstrap/src/main/java/org/apache/nifi/fn/NiFiFn.java new file mode 100644 index 0000000000..ab3f6bbe42 --- /dev/null +++ b/nifi-fn/nifi-fn-bootstrap/src/main/java/org/apache/nifi/fn/NiFiFn.java @@ -0,0 +1,185 @@ +/* + * 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.fn; + +import org.apache.nifi.nar.NarManifestEntry; +import org.apache.nifi.nar.NarUnpacker; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.File; +import java.io.FileNotFoundException; +import java.io.IOException; +import java.lang.reflect.InvocationTargetException; +import java.lang.reflect.Method; +import java.net.MalformedURLException; +import java.net.URL; +import java.net.URLClassLoader; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.concurrent.TimeUnit; +import java.util.jar.Attributes; +import java.util.jar.JarFile; +import java.util.jar.Manifest; + +public class NiFiFn { + private static final Logger logger = LoggerFactory.getLogger(NiFiFn.class); + + private static final String FN_CORE_NAR_ID = "nifi-fn-nar"; + + public static final String PROGRAM_CLASS_NAME = "org.apache.nifi.fn.runtimes.Program"; + + public static final String RUN_FROM_REGISTRY = "RunFromRegistry"; + public static final String RUN_YARN_SERVICE_FROM_REGISTRY = "RunYARNServiceFromRegistry"; + public static final String RUN_OPENWHISK_ACTION_SERVER = "RunOpenwhiskActionServer"; + + public static final String REGISTRY = "nifi_registry"; + public static final String BUCKETID = "nifi_bucket"; + public static final String FLOWID = "nifi_flow"; + public static final String CONTENT = "nifi_content"; + + public static void main(final String[] args) throws IOException, ClassNotFoundException, NoSuchMethodException, InvocationTargetException, IllegalAccessException { + if (args.length < 4) { + printUsage(); + return; + } + + final File libDir = new File(args[0]); + if (!libDir.exists()) { + System.out.println("Specified lib directory <" + libDir + "> does not exist"); + return; + } + + final File[] jarFiles = libDir.listFiles(file -> file.getName().endsWith(".jar")); + if (jarFiles == null) { + System.out.println("Could not obtain listing of NiFi-FN Lib directory <" + libDir + ">"); + return; + } + + final URL[] jarUrls = toURLs(jarFiles); + + final URLClassLoader rootClassLoader = new URLClassLoader(jarUrls); + + final File[] narFiles = libDir.listFiles(file -> file.getName().endsWith(".nar")); + if (narFiles == null) { + System.out.println("Could not obtain listing of NiFi-FN lib directory <" + libDir + ">"); + return; + } + + final File narWorkingDirectory = new File(args[1]); + if (!narWorkingDirectory.exists() && !narWorkingDirectory.mkdirs()) { + throw new IOException("Could not create NAR Working Directory <" + narWorkingDirectory + ">"); + } + + File fnCoreWorkingDirectory = null; + logger.info("Unpacking {} NARs", narFiles.length); + final long startUnpack = System.nanoTime(); + for (final File narFile : narFiles) { + final File unpackedDirectory = NarUnpacker.unpackNar(narFile, narWorkingDirectory); + if (isFnCoreNar(narFile)) { + fnCoreWorkingDirectory = unpackedDirectory; + } + } + final long millis = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startUnpack); + logger.info("Finished unpacking {} NARs in {} millis", narFiles.length, millis); + + + if (fnCoreWorkingDirectory == null) { + throw new FileNotFoundException("Could not find NiFi FN core NAR in the lib directory <" + libDir + ">"); + } + + final File bundledDependenciesDir = new File(fnCoreWorkingDirectory, NarUnpacker.BUNDLED_DEPENDENCIES_DIRECTORY); + final File[] fnCoreFiles = bundledDependenciesDir.listFiles(); + if (fnCoreFiles == null) { + throw new IOException("Could not obtain listing of NiFi-FN NAR's bundled dependencies in working directory <" + bundledDependenciesDir + ">"); + } + final URL[] fnCoreUrls = toURLs(fnCoreFiles); + + final URLClassLoader fnCoreClassLoader = new URLClassLoader(fnCoreUrls, rootClassLoader); + Thread.currentThread().setContextClassLoader(fnCoreClassLoader); + + final Class programClass = Class.forName(PROGRAM_CLASS_NAME, true, fnCoreClassLoader); + final Method launchMethod = programClass.getMethod("launch", String[].class, ClassLoader.class, File.class); + launchMethod.setAccessible(true); + + final String[] shiftedArgs = Arrays.copyOfRange(args, 2, args.length); + launchMethod.invoke(null, shiftedArgs, rootClassLoader, narWorkingDirectory); + } + + private static URL[] toURLs(final File[] files) throws MalformedURLException { + final List urls = new ArrayList<>(); + for (final File file : files) { + urls.add(file.toURI().toURL()); + } + + return urls.toArray(new URL[0]); + } + + private static boolean isFnCoreNar(final File file) throws IOException { + final String filename = file.getName(); + if (filename.startsWith("nifi-fn") && filename.endsWith(".nar")) { + try (final JarFile jarFile = new JarFile(file)) { + final Manifest manifest = jarFile.getManifest(); + + final Attributes attributes = manifest.getMainAttributes(); + final String narId = attributes.getValue(NarManifestEntry.NAR_ID.getManifestName()); + + return FN_CORE_NAR_ID.equals(narId); + } + } + + return false; + } + + private static void printUsage() { + System.out.println("Usage:"); + System.out.println(" 1) " + RUN_FROM_REGISTRY + " [Once|Continuous] " + + "[] []"); + System.out.println(" " + RUN_FROM_REGISTRY + " [Once|Continuous] --json "); + System.out.println(" " + RUN_FROM_REGISTRY + " [Once|Continuous] --file "); + System.out.println(); + System.out.println(" 2) " + RUN_YARN_SERVICE_FROM_REGISTRY + " <# of Containers> \\"); + System.out.println(" " + + "[] []"); + System.out.println(" " + RUN_YARN_SERVICE_FROM_REGISTRY + " <# of Containers> " + + "--json "); + System.out.println(" " + RUN_YARN_SERVICE_FROM_REGISTRY + " <# of Containers> " + + "--file "); + System.out.println(); + System.out.println(" 3) " + RUN_OPENWHISK_ACTION_SERVER + " "); + System.out.println(); + System.out.println("Examples:"); + System.out.println(" 1) " + RUN_FROM_REGISTRY + " ./lib ./work Once http://172.0.0.1:61080 e53b8a0d-5c85-4fcd-912a-1c549a586c83 6cf8277a-c402-4957-8623-0fa9890dd45d \\"); + System.out.println(" \"DestinationDirectory-/tmp/nififn/output2/\" \"\" \"absolute.path-/tmp/nififn/input/;filename-test.txt\" \"absolute.path-/tmp/nififn/input/;" + + "filename-test2.txt\""); + System.out.println(" 2) " + RUN_FROM_REGISTRY + " ./lib ./work Once http://172.0.0.1:61080 e53b8a0d-5c85-4fcd-912a-1c549a586c83 6cf8277a-c402-4957-8623-0fa9890dd45d \\"); + System.out.println(" \"DestinationDirectory-/tmp/nififn/output2/\" \"f25c9204-6c95-3aa9-b0a8-c556f5f61849\" \"absolute.path-/tmp/nififn/input/;filename-test.txt\""); + System.out.println(" 3) " + RUN_YARN_SERVICE_FROM_REGISTRY + " ./lib ./work http://127.0.0.1:8088 nifi-fn:latest kafka-to-solr 3 --file kafka-to-solr.json"); + System.out.println(" 4) " + RUN_OPENWHISK_ACTION_SERVER + " ./lib ./work 8080"); + System.out.println(); + System.out.println("Notes:"); + System.out.println(" 1) will be split on ';' and '-' then injected into the flow using the variable registry interface."); + System.out.println(" 2) will be split on ';'. FlowFiles routed to matching output ports will immediately fail the flow."); + System.out.println(" 3) will be split on ';' and '-' then injected into the flow using the \"" + CONTENT + "\" field as the FlowFile content."); + System.out.println(" 4) Multiple arguments can be provided."); + System.out.println(" 5) The configuration file must be in JSON format. "); + System.out.println(" 6) When providing configurations via JSON, the following attributes must be provided: " + REGISTRY + ", " + BUCKETID + ", " + FLOWID + "."); + System.out.println(" All other attributes will be passed to the flow using the variable registry interface"); + System.out.println(); + } +} diff --git a/nifi-fn/nifi-fn-core/pom.xml b/nifi-fn/nifi-fn-core/pom.xml new file mode 100644 index 0000000000..90eb9f4d7c --- /dev/null +++ b/nifi-fn/nifi-fn-core/pom.xml @@ -0,0 +1,148 @@ + + + + 4.0.0 + + + org.apache.nifi + nifi-fn + 1.10.0-SNAPSHOT + + + nifi-fn-core + + + + + org.apache.maven.plugins + maven-dependency-plugin + + + copy-dependencies + prepare-package + + copy-dependencies + + + ${project.build.directory}/nifi-fn-lib + false + true + true + compile + nar + + + + + + + + + + + org.apache.nifi + nifi-api + 1.10.0-SNAPSHOT + + + org.apache.nifi + nifi-framework-api + 1.10.0-SNAPSHOT + + + org.apache.nifi + nifi-framework-core-api + 1.10.0-SNAPSHOT + + + + org.apache.nifi + nifi-framework-nar-loading-utils + 1.10.0-SNAPSHOT + + + org.apache.nifi + nifi-nar-utils + 1.10.0-SNAPSHOT + + + org.apache.nifi + nifi-expression-language + 1.10.0-SNAPSHOT + + + org.apache.nifi + nifi-data-provenance-utils + 1.10.0-SNAPSHOT + + + com.google.code.gson + gson + 2.7 + + + org.slf4j + slf4j-api + 1.7.25 + + + + + + junit + junit + 4.12 + test + + + org.apache.nifi + nifi-standard-processors + 1.10.0-SNAPSHOT + test + + + org.apache.nifi + nifi-web-api + 1.10.0-SNAPSHOT + war + test + + + + \ No newline at end of file diff --git a/nifi-fn/nifi-fn-core/src/main/java/org/apache/nifi/fn/bootstrap/ExtensionDiscovery.java b/nifi-fn/nifi-fn-core/src/main/java/org/apache/nifi/fn/bootstrap/ExtensionDiscovery.java new file mode 100644 index 0000000000..e05bf944c8 --- /dev/null +++ b/nifi-fn/nifi-fn-core/src/main/java/org/apache/nifi/fn/bootstrap/ExtensionDiscovery.java @@ -0,0 +1,57 @@ +/* + * 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.fn.bootstrap; + +import org.apache.nifi.bundle.Bundle; +import org.apache.nifi.nar.ExtensionManager; +import org.apache.nifi.nar.NarClassLoaders; +import org.apache.nifi.nar.NarClassLoadersHolder; +import org.apache.nifi.nar.StandardExtensionDiscoveringManager; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.File; +import java.io.IOException; +import java.util.Set; +import java.util.concurrent.TimeUnit; + +public class ExtensionDiscovery { + private static final Logger logger = LoggerFactory.getLogger(ExtensionDiscovery.class); + + public static ExtensionManager discover(final File narWorkingDirectory, final ClassLoader systemClassLoader) throws IOException { + NarClassLoaders narClassLoaders = NarClassLoadersHolder.getInstance(); + + final long discoveryStart = System.nanoTime(); + try { + narClassLoaders.init(systemClassLoader, null, narWorkingDirectory); + } catch (final ClassNotFoundException cnfe) { + throw new IOException("Could not initialize Class Loaders", cnfe); + } + + final Set narBundles = narClassLoaders.getBundles(); + + final StandardExtensionDiscoveringManager extensionManager = new StandardExtensionDiscoveringManager(); + extensionManager.discoverExtensions(narBundles); + extensionManager.logClassLoaderMapping(); + + final long discoveryMillis = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - discoveryStart); + logger.info("Successfully discovered extensions in {} milliseconds", discoveryMillis); + + return extensionManager; + } + +} diff --git a/nifi-fn/nifi-fn-core/src/main/java/org/apache/nifi/fn/bootstrap/InMemoryFlowFile.java b/nifi-fn/nifi-fn-core/src/main/java/org/apache/nifi/fn/bootstrap/InMemoryFlowFile.java new file mode 100644 index 0000000000..e661f6446c --- /dev/null +++ b/nifi-fn/nifi-fn-core/src/main/java/org/apache/nifi/fn/bootstrap/InMemoryFlowFile.java @@ -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. + */ +package org.apache.nifi.fn.bootstrap; + +import org.apache.nifi.controller.repository.FlowFileRecord; + +public interface InMemoryFlowFile extends FlowFileRecord { + String toStringFull(); +} diff --git a/nifi-fn/nifi-fn-core/src/main/java/org/apache/nifi/fn/bootstrap/RunnableFlow.java b/nifi-fn/nifi-fn-core/src/main/java/org/apache/nifi/fn/bootstrap/RunnableFlow.java new file mode 100644 index 0000000000..51c8046389 --- /dev/null +++ b/nifi-fn/nifi-fn-core/src/main/java/org/apache/nifi/fn/bootstrap/RunnableFlow.java @@ -0,0 +1,27 @@ +/* + * 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.fn.bootstrap; + +import java.util.Queue; + +public interface RunnableFlow { + + boolean runOnce(Queue queue); + + boolean run(Queue queue); + +} diff --git a/nifi-fn/nifi-fn-core/src/main/java/org/apache/nifi/fn/bootstrap/RunnableFlowFactory.java b/nifi-fn/nifi-fn-core/src/main/java/org/apache/nifi/fn/bootstrap/RunnableFlowFactory.java new file mode 100644 index 0000000000..adc195a191 --- /dev/null +++ b/nifi-fn/nifi-fn-core/src/main/java/org/apache/nifi/fn/bootstrap/RunnableFlowFactory.java @@ -0,0 +1,108 @@ +/* + * 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.fn.bootstrap; + +import com.google.gson.JsonObject; +import com.google.gson.JsonParser; +import org.apache.nifi.controller.exception.ProcessorInstantiationException; +import org.apache.nifi.fn.core.FnFlow; +import org.apache.nifi.fn.core.RegistryUtil; +import org.apache.nifi.nar.ExtensionManager; +import org.apache.nifi.registry.VariableDescriptor; +import org.apache.nifi.registry.client.NiFiRegistryException; +import org.apache.nifi.registry.flow.VersionedFlowSnapshot; +import org.apache.nifi.registry.flow.VersionedProcessGroup; +import org.apache.nifi.reporting.InitializationException; + +import javax.net.ssl.SSLContext; +import java.io.File; +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Paths; +import java.security.NoSuchAlgorithmException; +import java.util.Arrays; +import java.util.HashMap; +import java.util.Map; + +public class RunnableFlowFactory { + + public static RunnableFlow fromJson(final String json) throws NiFiRegistryException, InitializationException, + IOException, ProcessorInstantiationException { + final JsonObject config = new JsonParser().parse(json).getAsJsonObject(); + return FnFlow.createAndEnqueueFromJSON(config); + } + + public static RunnableFlow fromJsonFile(final String filename, final ClassLoader systemClassLoader, final File narWorkingDir) throws IOException, + NiFiRegistryException, ProcessorInstantiationException, InitializationException { + final String json = new String(Files.readAllBytes(Paths.get(filename))); + final JsonObject config = new JsonParser().parse(json).getAsJsonObject(); + return FnFlow.createAndEnqueueFromJSON(config, systemClassLoader, narWorkingDir); + } + + public static RunnableFlow fromCommandLineArgs(final String[] args) throws InitializationException, IOException, ProcessorInstantiationException, NiFiRegistryException { + //Initialize flow + final String registryUrl = args[2]; + final String bucketID = args[3]; + final String flowID = args[4]; + final Map inputVariables = new HashMap<>(); + + if (args.length >= 6) { + final String[] variables = args[5].split(";"); + for (final String v : variables) { + String[] tokens = v.split("-"); + inputVariables.put(new VariableDescriptor(tokens[0]), tokens[1]); + } + } + + final String[] failureOutputPorts = args.length >= 7 ? args[6].split(";") : new String[]{}; + final SSLContext sslContext; + try { + sslContext = SSLContext.getDefault(); + } catch (NoSuchAlgorithmException e) { + throw new NiFiRegistryException("Could not get Default SSL Context", e); + } + + final VersionedFlowSnapshot snapshot = new RegistryUtil(registryUrl, sslContext).getFlowByID(bucketID, flowID); + final VersionedProcessGroup versionedFlow = snapshot.getFlowContents(); + + final ExtensionManager extensionManager = ExtensionDiscovery.discover(new File("./work"), ClassLoader.getSystemClassLoader()); + final FnFlow flow = new FnFlow(versionedFlow, extensionManager, () -> inputVariables, Arrays.asList(failureOutputPorts), true, sslContext); + + // Enqueue all provided flow files + if (7 < args.length) { + int i = 7; + while (i++ < args.length) { + final Map attributes = new HashMap<>(); + byte[] content = {}; + + final String[] attributesArr = args[i].split(";"); + for (final String v : attributesArr) { + final String[] tokens = v.split("-"); + if (tokens[0].equals(FnFlow.CONTENT)) { + content = tokens[1].getBytes(); + } else { + attributes.put(tokens[0], tokens[1]); + } + } + + flow.enqueueFlowFile(content, attributes); + } + } + + return flow; + } +} diff --git a/nifi-fn/nifi-fn-core/src/main/java/org/apache/nifi/fn/core/AbstractFnComponent.java b/nifi-fn/nifi-fn-core/src/main/java/org/apache/nifi/fn/core/AbstractFnComponent.java new file mode 100644 index 0000000000..5e2a2439a1 --- /dev/null +++ b/nifi-fn/nifi-fn-core/src/main/java/org/apache/nifi/fn/core/AbstractFnComponent.java @@ -0,0 +1,129 @@ +/* + * 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.fn.core; + +import org.apache.nifi.logging.ComponentLog; +import org.apache.nifi.processor.Relationship; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; + +public abstract class AbstractFnComponent implements FnComponent { + private List parents = new ArrayList<>(); + private List incomingConnections = new ArrayList<>(); + private final Map> children = new HashMap<>(); + private final Set autoTermination = new HashSet<>(); + private final Set successOutputPorts = new HashSet<>(); + private final Set failureOutputPorts = new HashSet<>(); + + + public AbstractFnComponent() { + + } + + public List getParents() { + return Collections.unmodifiableList(parents); + } + + public void addParent(final FnComponent parent) { + if (parent != null) { + parents.add(parent); + } + } + + public void addIncomingConnection(final String connectionId) { + this.incomingConnections.add(connectionId); + } + + public void addOutputPort(Relationship relationship, boolean isFailurePort) { + if (isFailurePort) { + this.failureOutputPorts.add(relationship); + } else { + this.successOutputPorts.add(relationship); + } + } + + public void addChild(FnComponent child, Relationship relationship) { + List list = children.computeIfAbsent(relationship, r -> new ArrayList<>()); + list.add(child); + + getContext().addConnection(relationship); + } + + public void addAutoTermination(Relationship relationship) { + this.autoTermination.add(relationship); + getContext().addConnection(relationship); + } + + + public boolean validate() { + if (!getContext().isValid()) { + return false; + } + + for (final Relationship relationship : getRelationships()) { + boolean hasChildren = this.children.containsKey(relationship); + boolean hasAutoterminate = this.autoTermination.contains(relationship); + boolean hasFailureOutputPort = this.failureOutputPorts.contains(relationship); + boolean hasSuccessOutputPort = this.successOutputPorts.contains(relationship); + + if (!(hasChildren || hasAutoterminate || hasFailureOutputPort || hasSuccessOutputPort)) { + getLogger().error("Component: {}, Relationship: {}, needs either auto terminate, child processors, or an output port", new Object[] {toString(), relationship.getName()}); + return false; + } + } + + for (final Map.Entry> entry : this.children.entrySet()) { + for (final FnComponent component : entry.getValue()) { + if (!component.validate()) { + return false; + } + } + } + + return true; + } + + protected Map> getChildren() { + return children; + } + + protected Set getSuccessOutputPorts() { + return successOutputPorts; + } + + protected Set getFailureOutputPorts() { + return failureOutputPorts; + } + + protected boolean isAutoTerminated(final Relationship relationship) { + return autoTermination.contains(relationship); + } + + + + public abstract Set getRelationships(); + + protected abstract FnConnectionContext getContext(); + + protected abstract ComponentLog getLogger(); +} diff --git a/nifi-fn/nifi-fn-core/src/main/java/org/apache/nifi/fn/core/ComponentFactory.java b/nifi-fn/nifi-fn-core/src/main/java/org/apache/nifi/fn/core/ComponentFactory.java new file mode 100644 index 0000000000..c724643166 --- /dev/null +++ b/nifi-fn/nifi-fn-core/src/main/java/org/apache/nifi/fn/core/ComponentFactory.java @@ -0,0 +1,212 @@ +/* + * 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.fn.core; + +import org.apache.commons.lang3.StringUtils; +import org.apache.nifi.attribute.expression.language.StandardPropertyValue; +import org.apache.nifi.bundle.Bundle; +import org.apache.nifi.bundle.BundleCoordinate; +import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.controller.ControllerService; +import org.apache.nifi.controller.exception.ControllerServiceInstantiationException; +import org.apache.nifi.controller.exception.ProcessorInstantiationException; +import org.apache.nifi.logging.ComponentLog; +import org.apache.nifi.nar.ExtensionManager; +import org.apache.nifi.processor.Processor; +import org.apache.nifi.processor.ProcessorInitializationContext; +import org.apache.nifi.processor.Relationship; +import org.apache.nifi.registry.VariableRegistry; +import org.apache.nifi.registry.flow.VersionedControllerService; +import org.apache.nifi.registry.flow.VersionedProcessor; +import org.apache.nifi.util.file.classloader.ClassLoaderUtils; + +import java.net.MalformedURLException; +import java.net.URL; +import java.util.Arrays; +import java.util.Collections; +import java.util.LinkedHashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; + +public class ComponentFactory { + private final ExtensionManager extensionManager; + + public ComponentFactory(final ExtensionManager extensionManager) { + this.extensionManager = extensionManager; + } + + + public FnProcessorWrapper createProcessor(final VersionedProcessor versionedProcessor, final boolean materializeContent, final FnControllerServiceLookup controllerServiceLookup, + final VariableRegistry variableRegistry, final Set classpathUrls) throws ProcessorInstantiationException { + final String type = versionedProcessor.getType(); + final String identifier = versionedProcessor.getIdentifier(); + + final Bundle bundle = getAvailableBundle(versionedProcessor.getBundle(), type); + if (bundle == null) { + throw new IllegalStateException("Unable to find bundle for coordinate " + + versionedProcessor.getBundle().getGroup() + ":" + + versionedProcessor.getBundle().getArtifact() + ":" + + versionedProcessor.getBundle().getVersion()); + } + + final ClassLoader ctxClassLoader = Thread.currentThread().getContextClassLoader(); + try { + final ClassLoader detectedClassLoader = extensionManager.createInstanceClassLoader(type, identifier, bundle, + classpathUrls == null ? Collections.emptySet() : classpathUrls); + + System.out.println("Setting context class loader to " + detectedClassLoader + " (parent = " + detectedClassLoader.getParent() + ") to create " + type); + final Class rawClass = Class.forName(type, true, detectedClassLoader); + Thread.currentThread().setContextClassLoader(detectedClassLoader); + + final Object extensionInstance = rawClass.newInstance(); + final ComponentLog componentLog = new SLF4JComponentLog(extensionInstance); + + final Processor processor = (Processor) extensionInstance; + final ProcessorInitializationContext initializationContext = new FnProcessorInitializationContext(versionedProcessor.getIdentifier(), processor, controllerServiceLookup); + processor.initialize(initializationContext); + + // If no classpath urls were provided, check if we need to add additional classpath URL's based on configured properties. + if (classpathUrls == null) { + final Set additionalClasspathUrls = getAdditionalClasspathResources(processor.getPropertyDescriptors(), processor.getIdentifier(), versionedProcessor.getProperties(), + variableRegistry,componentLog); + + if (!additionalClasspathUrls.isEmpty()) { + return createProcessor(versionedProcessor, materializeContent, controllerServiceLookup, variableRegistry, additionalClasspathUrls); + } + } + + final FnProcessorWrapper processorWrapper = new FnProcessorWrapper(versionedProcessor.getIdentifier(), processor, null, + controllerServiceLookup, variableRegistry, materializeContent, detectedClassLoader); + + // Configure the Processor + processorWrapper.setAnnotationData(versionedProcessor.getAnnotationData()); + versionedProcessor.getProperties().forEach(processorWrapper::setProperty); + for (String relationship : versionedProcessor.getAutoTerminatedRelationships()) { + processorWrapper.addAutoTermination(new Relationship.Builder().name(relationship).build()); + } + + return processorWrapper; + } catch (final Exception e) { + throw new ProcessorInstantiationException(type, e); + } finally { + if (ctxClassLoader != null) { + Thread.currentThread().setContextClassLoader(ctxClassLoader); + } + } + } + + + private Set getAdditionalClasspathResources(final List propertyDescriptors, final String componentId, final Map properties, + final VariableRegistry variableRegistry, final ComponentLog logger) { + final Set modulePaths = new LinkedHashSet<>(); + for (final PropertyDescriptor descriptor : propertyDescriptors) { + if (descriptor.isDynamicClasspathModifier()) { + final String value = properties.get(descriptor.getName()); + if (!StringUtils.isEmpty(value)) { + final StandardPropertyValue propertyValue = new StandardPropertyValue(value, null, variableRegistry); + modulePaths.add(propertyValue.evaluateAttributeExpressions().getValue()); + } + } + } + + final Set additionalUrls = new LinkedHashSet<>(); + try { + final URL[] urls = ClassLoaderUtils.getURLsForClasspath(modulePaths, null, true); + if (urls != null) { + additionalUrls.addAll(Arrays.asList(urls)); + } + } catch (MalformedURLException mfe) { + logger.error("Error processing classpath resources for " + componentId + ": " + mfe.getMessage(), mfe); + } + + return additionalUrls; + } + + + public ControllerService createControllerService(final VersionedControllerService versionedControllerService, final VariableRegistry variableRegistry) { + return createControllerService(versionedControllerService, variableRegistry, null); + } + + + private ControllerService createControllerService(final VersionedControllerService versionedControllerService, final VariableRegistry variableRegistry, final Set classpathUrls) { + + final String type = versionedControllerService.getType(); + final String identifier = versionedControllerService.getIdentifier(); + + final Bundle bundle = getAvailableBundle(versionedControllerService.getBundle(), type); + if (bundle == null) { + throw new IllegalStateException("Unable to find bundle for coordinate " + + versionedControllerService.getBundle().getGroup() + ":" + + versionedControllerService.getBundle().getArtifact() + ":" + + versionedControllerService.getBundle().getVersion()); + } + + final ClassLoader ctxClassLoader = Thread.currentThread().getContextClassLoader(); + try { + final ClassLoader detectedClassLoader = extensionManager.createInstanceClassLoader(type, identifier, bundle, + classpathUrls == null ? Collections.emptySet() : classpathUrls); + + System.out.println("Setting context class loader to " + detectedClassLoader + " (parent = " + detectedClassLoader.getParent() + ") to create " + type); + final Class rawClass = Class.forName(type, true, detectedClassLoader); + Thread.currentThread().setContextClassLoader(detectedClassLoader); + + final Object extensionInstance = rawClass.newInstance(); + final ComponentLog componentLog = new SLF4JComponentLog(extensionInstance); + + final ControllerService service = (ControllerService) extensionInstance; + + // If no classpath urls were provided, check if we need to add additional classpath URL's based on configured properties. + if (classpathUrls == null) { + final Set additionalClasspathUrls = getAdditionalClasspathResources(service.getPropertyDescriptors(), service.getIdentifier(), versionedControllerService.getProperties(), + variableRegistry, componentLog); + + if (!additionalClasspathUrls.isEmpty()) { + return createControllerService(versionedControllerService, variableRegistry, additionalClasspathUrls); + } + } + + return service; + } catch (final Exception e) { + throw new ControllerServiceInstantiationException(type, e); + } finally { + if (ctxClassLoader != null) { + Thread.currentThread().setContextClassLoader(ctxClassLoader); + } + } + } + + private Bundle getAvailableBundle(final org.apache.nifi.registry.flow.Bundle bundle, final String componentType) { + final BundleCoordinate bundleCoordinate = new BundleCoordinate(bundle.getGroup(), bundle.getArtifact(), bundle.getVersion()); + final Bundle availableBundle = extensionManager.getBundle(bundleCoordinate); + if (availableBundle != null) { + return availableBundle; + } + + final List possibleBundles = extensionManager.getBundles(componentType); + if (possibleBundles.isEmpty()) { + throw new IllegalStateException("Could not find any NiFi Bundles that contain the Extension [" + componentType + "]"); + } + + if (possibleBundles.size() > 1) { + throw new IllegalStateException("Found " + possibleBundles.size() + " different NiFi Bundles that contain the Extension [" + componentType + "] but none of them had a version of " + + bundle.getVersion()); + } + + return possibleBundles.get(0); + } +} diff --git a/nifi-fn/nifi-fn-core/src/main/java/org/apache/nifi/fn/core/FnComponent.java b/nifi-fn/nifi-fn-core/src/main/java/org/apache/nifi/fn/core/FnComponent.java new file mode 100644 index 0000000000..ef55896e1f --- /dev/null +++ b/nifi-fn/nifi-fn-core/src/main/java/org/apache/nifi/fn/core/FnComponent.java @@ -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. + */ +package org.apache.nifi.fn.core; + +import org.apache.nifi.fn.bootstrap.InMemoryFlowFile; +import org.apache.nifi.processor.Relationship; + +import java.util.List; +import java.util.Queue; + +public interface FnComponent { + + void shutdown(); + + void enqueueAll(Queue list); + + boolean runRecursive(Queue queue); + + boolean validate(); + + void addIncomingConnection(String connectionId); + + void addOutputPort(Relationship relationship, boolean isFailurePort); + + boolean isMaterializeContent(); + + List getParents(); + + void addParent(FnComponent parent); + + void addChild(FnComponent child, Relationship relationship); +} diff --git a/nifi-fn/src/main/java/org/apache/nifi/fn/core/FnConfigurationContext.java b/nifi-fn/nifi-fn-core/src/main/java/org/apache/nifi/fn/core/FnConfigurationContext.java similarity index 97% rename from nifi-fn/src/main/java/org/apache/nifi/fn/core/FnConfigurationContext.java rename to nifi-fn/nifi-fn-core/src/main/java/org/apache/nifi/fn/core/FnConfigurationContext.java index 7c803598d8..8bddc71f03 100644 --- a/nifi-fn/src/main/java/org/apache/nifi/fn/core/FnConfigurationContext.java +++ b/nifi-fn/nifi-fn-core/src/main/java/org/apache/nifi/fn/core/FnConfigurationContext.java @@ -61,7 +61,7 @@ public class FnConfigurationContext implements ConfigurationContext { @Override public Map getAllProperties() { - final Map propValueMap = new LinkedHashMap<>(); + final Map propValueMap = new LinkedHashMap<>(); for (final Map.Entry entry : getProperties().entrySet()) { propValueMap.put(entry.getKey().getName(), entry.getValue()); } diff --git a/nifi-fn/nifi-fn-core/src/main/java/org/apache/nifi/fn/core/FnConnectionContext.java b/nifi-fn/nifi-fn-core/src/main/java/org/apache/nifi/fn/core/FnConnectionContext.java new file mode 100644 index 0000000000..a62b02e762 --- /dev/null +++ b/nifi-fn/nifi-fn-core/src/main/java/org/apache/nifi/fn/core/FnConnectionContext.java @@ -0,0 +1,26 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.fn.core; + +import org.apache.nifi.processor.Relationship; + +public interface FnConnectionContext { + + void addConnection(Relationship relationship); + + boolean isValid(); +} diff --git a/nifi-fn/src/main/java/org/apache/nifi/fn/core/FnControllerServiceConfiguration.java b/nifi-fn/nifi-fn-core/src/main/java/org/apache/nifi/fn/core/FnControllerServiceConfiguration.java similarity index 97% rename from nifi-fn/src/main/java/org/apache/nifi/fn/core/FnControllerServiceConfiguration.java rename to nifi-fn/nifi-fn-core/src/main/java/org/apache/nifi/fn/core/FnControllerServiceConfiguration.java index 2d2cec4b7d..0e5c24a6e2 100644 --- a/nifi-fn/src/main/java/org/apache/nifi/fn/core/FnControllerServiceConfiguration.java +++ b/nifi-fn/nifi-fn-core/src/main/java/org/apache/nifi/fn/core/FnControllerServiceConfiguration.java @@ -50,7 +50,10 @@ public class FnControllerServiceConfiguration { public void setProperties(final Map props) { this.properties = new HashMap<>(props); } - public void setProperty(final PropertyDescriptor key, final String value) { this.properties.put(key,value); } + + public void setProperty(final PropertyDescriptor key, final String value) { + this.properties.put(key, value); + } public String getProperty(final PropertyDescriptor descriptor) { final String value = properties.get(descriptor); diff --git a/nifi-fn/src/main/java/org/apache/nifi/fn/core/FnControllerServiceLookup.java b/nifi-fn/nifi-fn-core/src/main/java/org/apache/nifi/fn/core/FnControllerServiceLookup.java similarity index 70% rename from nifi-fn/src/main/java/org/apache/nifi/fn/core/FnControllerServiceLookup.java rename to nifi-fn/nifi-fn-core/src/main/java/org/apache/nifi/fn/core/FnControllerServiceLookup.java index 8b98550938..2f7ec980ac 100644 --- a/nifi-fn/src/main/java/org/apache/nifi/fn/core/FnControllerServiceLookup.java +++ b/nifi-fn/nifi-fn-core/src/main/java/org/apache/nifi/fn/core/FnControllerServiceLookup.java @@ -17,9 +17,7 @@ package org.apache.nifi.fn.core; import org.apache.nifi.annotation.lifecycle.OnAdded; -import org.apache.nifi.annotation.lifecycle.OnDisabled; import org.apache.nifi.annotation.lifecycle.OnEnabled; -import org.apache.nifi.annotation.lifecycle.OnRemoved; import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.components.ValidationContext; import org.apache.nifi.components.ValidationResult; @@ -27,36 +25,30 @@ import org.apache.nifi.controller.ConfigurationContext; import org.apache.nifi.controller.ControllerService; import org.apache.nifi.controller.ControllerServiceLookup; import org.apache.nifi.registry.VariableRegistry; -import org.apache.nifi.registry.flow.VersionedControllerService; import org.apache.nifi.reporting.InitializationException; import java.lang.reflect.InvocationTargetException; -import java.util.*; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; import java.util.concurrent.ConcurrentHashMap; import static java.util.Objects.requireNonNull; public class FnControllerServiceLookup implements ControllerServiceLookup { - private final Map controllerServiceMap = new ConcurrentHashMap<>(); private final Map controllerServiceLoggers = new HashMap<>(); private final Map controllerServiceStateManagers = new HashMap<>(); - public Map getControllerServices() { return controllerServiceMap; } - public void addControllerService(final VersionedControllerService versionedControllerService) throws InitializationException { - String id = versionedControllerService.getIdentifier(); - ControllerService service = ReflectionUtils.createControllerService(versionedControllerService); - Map properties = versionedControllerService.getProperties(); - addControllerService(id,service,properties); - - } - public void addControllerService(final String identifier, final ControllerService service, final Map properties) throws InitializationException { + public void addControllerService(final ControllerService service) throws InitializationException { + final String identifier = service.getIdentifier(); final SLF4JComponentLog logger = new SLF4JComponentLog(service); controllerServiceLoggers.put(identifier, logger); @@ -66,11 +58,6 @@ public class FnControllerServiceLookup implements ControllerServiceLookup { final FnProcessContext initContext = new FnProcessContext(requireNonNull(service), this, requireNonNull(identifier), logger, serviceStateManager); service.initialize(initContext); - final Map resolvedProps = new HashMap<>(); - for (final Map.Entry entry : properties.entrySet()) { - resolvedProps.put(service.getPropertyDescriptor(entry.getKey()), entry.getValue()); - } - try { ReflectionUtils.invokeMethodsWithAnnotation(OnAdded.class, service); } catch (final InvocationTargetException | IllegalAccessException | IllegalArgumentException e) { @@ -81,20 +68,6 @@ public class FnControllerServiceLookup implements ControllerServiceLookup { controllerServiceMap.put(identifier, config); } - public void removeControllerService(final ControllerService service) throws InvocationTargetException, IllegalAccessException { - final ControllerService canonical = getControllerService(service.getIdentifier()); - - disableControllerService(canonical); - - ReflectionUtils.invokeMethodsWithAnnotation(OnRemoved.class, canonical); - - if (canonical == null || canonical != service) { - throw new IllegalArgumentException("Controller Service " + service + " is not known"); - } - - controllerServiceMap.remove(service.getIdentifier()); - } - protected FnControllerServiceConfiguration getConfiguration(final String identifier) { return controllerServiceMap.get(identifier); @@ -143,22 +116,6 @@ public class FnControllerServiceLookup implements ControllerServiceLookup { return status == null ? null : serviceIdentifier; } - - public void disableControllerService(final ControllerService service) throws InvocationTargetException, IllegalAccessException { - final FnControllerServiceConfiguration configuration = getConfiguration(service.getIdentifier()); - if (configuration == null) { - throw new IllegalArgumentException("Controller Service " + service + " is not known"); - } - - if (!configuration.isEnabled()) { - throw new IllegalStateException("Controller service " + service + " cannot be disabled because it is not enabled"); - } - - ReflectionUtils.invokeMethodsWithAnnotation(OnDisabled.class, service); - - - configuration.setEnabled(false); - } public void enableControllerService(final ControllerService service, VariableRegistry registry) throws InvocationTargetException, IllegalAccessException { final FnControllerServiceConfiguration configuration = getConfiguration(service.getIdentifier()); if (configuration == null) { @@ -171,28 +128,15 @@ public class FnControllerServiceLookup implements ControllerServiceLookup { final ConfigurationContext configContext = new FnConfigurationContext(service, configuration.getProperties(), this, registry); ReflectionUtils.invokeMethodsWithAnnotation(OnEnabled.class, service, configContext); - configuration.setEnabled(true); } - public SLF4JComponentLog getControllerServiceLogger(final String identifier) { - return controllerServiceLoggers.get(identifier); - } - - Map getControllerServiceProperties(final ControllerService controllerService) { - return this.getConfiguration(controllerService.getIdentifier()).getProperties(); - } - String getControllerServiceAnnotationData(final ControllerService controllerService) { - return this.getConfiguration(controllerService.getIdentifier()).getAnnotationData(); - } - public FnStateManager getStateManager(final ControllerService controllerService) { - return controllerServiceStateManagers.get(controllerService.getIdentifier()); - } public void setControllerServiceAnnotationData(final ControllerService service, final String annotationData) { final FnControllerServiceConfiguration configuration = getControllerServiceConfigToUpdate(service); configuration.setAnnotationData(annotationData); } + private FnControllerServiceConfiguration getControllerServiceConfigToUpdate(final ControllerService service) { final FnControllerServiceConfiguration configuration = getConfiguration(service.getIdentifier()); if (configuration == null) { @@ -205,7 +149,9 @@ public class FnControllerServiceLookup implements ControllerServiceLookup { return configuration; } - public ValidationResult setControllerServiceProperty(final ControllerService service, final PropertyDescriptor property, final FnProcessContext context, final VariableRegistry registry, final String value) { + + public ValidationResult setControllerServiceProperty(final ControllerService service, final PropertyDescriptor property, final FnProcessContext context, final VariableRegistry registry, final + String value) { final FnStateManager serviceStateManager = controllerServiceStateManagers.get(service.getIdentifier()); if (serviceStateManager == null) { throw new IllegalStateException("Controller service " + service + " has not been added to this TestRunner via the #addControllerService method"); @@ -216,7 +162,7 @@ public class FnControllerServiceLookup implements ControllerServiceLookup { final FnControllerServiceConfiguration configuration = getControllerServiceConfigToUpdate(service); final String oldValue = configuration.getProperties().get(property); - configuration.setProperty(property,value); + configuration.setProperty(property, value); if ((value == null && oldValue != null) || (value != null && !value.equals(oldValue))) { service.onPropertyModified(property, oldValue, value); @@ -225,8 +171,4 @@ public class FnControllerServiceLookup implements ControllerServiceLookup { return validationResult; } - - - - } diff --git a/nifi-fn/nifi-fn-core/src/main/java/org/apache/nifi/fn/core/FnFlow.java b/nifi-fn/nifi-fn-core/src/main/java/org/apache/nifi/fn/core/FnFlow.java new file mode 100644 index 0000000000..59e237b2bc --- /dev/null +++ b/nifi-fn/nifi-fn-core/src/main/java/org/apache/nifi/fn/core/FnFlow.java @@ -0,0 +1,450 @@ +/* + * 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.fn.core; + +import com.google.gson.JsonArray; +import com.google.gson.JsonElement; +import com.google.gson.JsonObject; +import org.apache.nifi.controller.ControllerService; +import org.apache.nifi.controller.exception.ProcessorInstantiationException; +import org.apache.nifi.fn.bootstrap.ExtensionDiscovery; +import org.apache.nifi.fn.bootstrap.InMemoryFlowFile; +import org.apache.nifi.fn.bootstrap.RunnableFlow; +import org.apache.nifi.nar.ExtensionManager; +import org.apache.nifi.processor.Relationship; +import org.apache.nifi.registry.VariableDescriptor; +import org.apache.nifi.registry.VariableRegistry; +import org.apache.nifi.registry.client.NiFiRegistryException; +import org.apache.nifi.registry.flow.ConnectableComponent; +import org.apache.nifi.registry.flow.VersionedConnection; +import org.apache.nifi.registry.flow.VersionedControllerService; +import org.apache.nifi.registry.flow.VersionedFlowSnapshot; +import org.apache.nifi.registry.flow.VersionedPort; +import org.apache.nifi.registry.flow.VersionedProcessGroup; +import org.apache.nifi.registry.flow.VersionedProcessor; +import org.apache.nifi.registry.flow.VersionedRemoteGroupPort; +import org.apache.nifi.registry.flow.VersionedRemoteProcessGroup; +import org.apache.nifi.reporting.InitializationException; +import org.apache.nifi.security.util.SslContextFactory; + +import javax.net.ssl.SSLContext; +import java.io.File; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; +import java.util.Queue; +import java.util.Set; +import java.util.stream.Collectors; + +public class FnFlow implements RunnableFlow { + + public static final String REGISTRY = "registryUrl"; + public static final String BUCKETID = "bucketId"; + public static final String FLOWID = "flowId"; + public static final String FLOWVERSION = "flowVersion"; + public static final String MATERIALIZECONTENT = "materializeContent"; + public static final String FAILUREPORTS = "failurePortIds"; + public static final String FLOWFILES = "flowFiles"; + public static final String CONTENT = "nifi_content"; + public static final String VARIABLES = "variables"; + + public static final String SSL = "ssl"; + public static final String KEYSTORE = "keystore"; + public static final String KEYSTORE_PASS = "keystorePass"; + public static final String KEY_PASS = "keyPass"; + public static final String KEYSTORE_TYPE = "keystoreType"; + public static final String TRUSTSTORE = "truststore"; + public static final String TRUSTSTORE_PASS = "truststorePass"; + public static final String TRUSTSTORE_TYPE = "truststoreType"; + + private static final String DEFAULT_WORKING_DIR = "./work"; + + private List roots; + private volatile boolean stopRequested = false; + private FnComponent sourceComponent = null; + + private final ComponentFactory componentFactory; + + public FnFlow(final FnProcessorWrapper root) { + this(Collections.singletonList(root)); + } + + public FnFlow(final List roots) { + this.roots = roots; + this.componentFactory = null; + } + + + public FnFlow(final VersionedProcessGroup flow, final ExtensionManager extensionManager, final VariableRegistry variableRegistry, + final List failureOutputPorts, final boolean materializeContent, final SSLContext sslContext) throws ProcessorInstantiationException, InitializationException { + + this.componentFactory = new ComponentFactory(extensionManager); + + final Map processors = findProcessorsRecursive(flow).stream() + .collect(Collectors.toMap(VersionedProcessor::getIdentifier, proc -> proc)); + + final Map rpgs = new HashMap<>(); + final Map remotePorts = new HashMap<>(); + findRemoteGroupRecursive(flow, rpgs, remotePorts); + + final Set connections = findConnectionsRecursive(flow); + final Set inputPorts = flow.getInputPorts(); + + if (inputPorts.size() > 1) { + throw new IllegalArgumentException("Only one input port per flow is allowed"); + } + + final FnControllerServiceLookup serviceLookup = new FnControllerServiceLookup(); + + final Set controllerServices = flow.getControllerServices(); + for (final VersionedControllerService versionedControllerService : controllerServices) { + final ControllerService service = componentFactory.createControllerService(versionedControllerService, variableRegistry); + serviceLookup.addControllerService(service); + } + + final Map componentMap = new HashMap<>(); + + for (final VersionedConnection connection : connections) { + boolean isInputPortConnection = false; + + final ConnectableComponent source = connection.getSource(); + final ConnectableComponent destination = connection.getDestination(); + + FnComponent sourceComponent = null; + if (componentMap.containsKey(source.getId())) { + sourceComponent = componentMap.get(source.getId()); + } else { + switch (source.getType()) { + case PROCESSOR: + final VersionedProcessor processor = processors.get(source.getId()); + + if (processor == null) { + throw new IllegalArgumentException("Unknown input processor. " + source.getId()); + } else { + sourceComponent = componentFactory.createProcessor(processor, materializeContent, serviceLookup, variableRegistry, null); + componentMap.put(source.getId(), sourceComponent); + } + break; + case REMOTE_INPUT_PORT: + throw new IllegalArgumentException("Unsupported source type: " + source.getType()); + case REMOTE_OUTPUT_PORT: + final VersionedRemoteGroupPort remotePort = remotePorts.get(source.getId()); + final VersionedRemoteProcessGroup rpg = rpgs.get(remotePort.getRemoteGroupId()); + + sourceComponent = new FnRemoteOutputPort(rpg, remotePort, sslContext); + componentMap.put(source.getId(), sourceComponent); + break; + case OUTPUT_PORT: + case FUNNEL: + sourceComponent = new FnPassThroughComponent(); + componentMap.put(source.getId(), sourceComponent); + break; + case INPUT_PORT: + if (flow.getIdentifier().equals(connection.getGroupIdentifier())) { + isInputPortConnection = true; + } else { + sourceComponent = new FnPassThroughComponent(); + componentMap.put(source.getId(), sourceComponent); + } + + break; + } + } + + FnComponent destinationComponent = null; + switch (destination.getType()) { + case PROCESSOR: + if (componentMap.containsKey(destination.getId())) { + destinationComponent = componentMap.get(destination.getId()); + } else { + final VersionedProcessor processor = processors.get(destination.getId()); + if (processor == null) { + return; + } + + destinationComponent = componentFactory.createProcessor(processor, materializeContent, serviceLookup, variableRegistry, null); + destinationComponent.addParent(sourceComponent); + componentMap.put(destination.getId(), destinationComponent); + } + + break; + case REMOTE_INPUT_PORT: + if (componentMap.containsKey(destination.getId())) { + destinationComponent = componentMap.get(destination.getId()); + } else { + final VersionedRemoteGroupPort remotePort = remotePorts.get(destination.getId()); + final VersionedRemoteProcessGroup rpg = rpgs.get(remotePort.getRemoteGroupId()); + + destinationComponent = new FnRemoteInputPort(rpg, remotePort, sslContext); + destinationComponent.addParent(sourceComponent); + componentMap.put(destination.getId(), destinationComponent); + } + + break; + case REMOTE_OUTPUT_PORT: + throw new IllegalArgumentException("Unsupported destination type: " + destination.getType()); + case OUTPUT_PORT: + if (isInputPortConnection) { + throw new IllegalArgumentException("Input ports can not be mapped directly to output ports..."); + } + + // If Output Port is top-level port, treat it differently than if it's an inner group. + if (flow.getIdentifier().equals(connection.getGroupIdentifier())) { + //Link source and destination + for (final String selectedRelationship : connection.getSelectedRelationships()) { + final Relationship relationship = new Relationship.Builder().name(selectedRelationship).build(); + final boolean failurePort = failureOutputPorts.contains(destination.getId()); + sourceComponent.addOutputPort(relationship, failurePort); + } + + break; + } + + // Intentionally let the flow drop-through, and treat the same as an output port or funnel. + case INPUT_PORT: + case FUNNEL: + if (componentMap.containsKey(destination.getId())) { + destinationComponent = componentMap.get(destination.getId()); + } else { + destinationComponent = new FnPassThroughComponent(); + componentMap.put(destination.getId(), destinationComponent); + } + + break; + } + + if (destinationComponent != null) { + destinationComponent.addIncomingConnection(connection.getIdentifier()); + + if (isInputPortConnection) { + this.sourceComponent = destinationComponent; + } else { + destinationComponent.addParent(sourceComponent); + + //Link source and destination + for (final String relationship : connection.getSelectedRelationships()) { + sourceComponent.addChild(destinationComponent, new Relationship.Builder().name(relationship).build()); + } + } + + } + } + + roots = componentMap.entrySet() + .stream() + .filter(e -> e.getValue().getParents().isEmpty()) + .map(Map.Entry::getValue) + .collect(Collectors.toList()); + } + + private Set findProcessorsRecursive(final VersionedProcessGroup group) { + final Set processors = new HashSet<>(); + findProcessorsRecursive(group, processors); + return processors; + } + + private void findProcessorsRecursive(final VersionedProcessGroup group, final Set processors) { + processors.addAll(group.getProcessors()); + group.getProcessGroups().forEach(child -> findProcessorsRecursive(child, processors)); + } + + private Set findConnectionsRecursive(final VersionedProcessGroup group) { + final Set connections = new HashSet<>(); + findConnectionsRecursive(group, connections); + return connections; + } + + private void findConnectionsRecursive(final VersionedProcessGroup group, final Set connections) { + connections.addAll(group.getConnections()); + group.getProcessGroups().forEach(child -> findConnectionsRecursive(child, connections)); + } + + private void findRemoteGroupRecursive(final VersionedProcessGroup group, final Map rpgs, final Map ports) { + for (final VersionedRemoteProcessGroup rpg : group.getRemoteProcessGroups()) { + rpgs.put(rpg.getIdentifier(), rpg); + + rpg.getInputPorts().forEach(port -> ports.put(port.getIdentifier(), port)); + rpg.getOutputPorts().forEach(port -> ports.put(port.getIdentifier(), port)); + } + } + + + + public boolean run(final Queue output) { + while (!this.stopRequested) { + for (final FnComponent pw : roots) { + final boolean successful = pw.runRecursive(output); + if (!successful) { + return false; + } + } + } + + return true; + } + + public boolean runOnce(Queue output) { + for (final FnComponent pw : roots) { + final boolean successful = pw.runRecursive(output); + if (!successful) { + return false; + } + } + + return true; + } + + public void shutdown() { + this.stopRequested = true; + this.roots.forEach(FnComponent::shutdown); + } + + public static SSLContext getSSLContext(final JsonObject config) { + if (!config.has(SSL)) { + return null; + } + + final JsonObject sslObject = config.get(SSL).getAsJsonObject(); + if (sslObject.has(KEYSTORE) && sslObject.has(KEYSTORE_PASS) && sslObject.has(KEYSTORE_TYPE) + && sslObject.has(TRUSTSTORE) && sslObject.has(TRUSTSTORE_PASS) && sslObject.has(TRUSTSTORE_TYPE)) { + + final String keystore = sslObject.get(KEYSTORE).getAsString(); + final String keystorePass = sslObject.get(KEYSTORE_PASS).getAsString(); + final String keyPass = sslObject.has(KEY_PASS) ? sslObject.get(KEY_PASS).getAsString() : keystorePass; + final String keystoreType = sslObject.get(KEYSTORE_TYPE).getAsString(); + + final String truststore = sslObject.get(TRUSTSTORE).getAsString(); + final String truststorePass = sslObject.get(TRUSTSTORE_PASS).getAsString(); + final String truststoreType = sslObject.get(TRUSTSTORE_TYPE).getAsString(); + + try { + return SslContextFactory.createSslContext(keystore, keystorePass.toCharArray(), keyPass.toCharArray(), keystoreType, + truststore, truststorePass.toCharArray(), truststoreType, SslContextFactory.ClientAuth.REQUIRED, "TLS"); + } catch (final Exception e) { + throw new RuntimeException("Failed to create Keystore", e); + } + } + + return null; + } + + public static FnFlow createAndEnqueueFromJSON(final JsonObject args) throws InitializationException, IOException, ProcessorInstantiationException, NiFiRegistryException { + return createAndEnqueueFromJSON(args, ClassLoader.getSystemClassLoader(), new File(DEFAULT_WORKING_DIR)); + } + + public static FnFlow createAndEnqueueFromJSON(final JsonObject args, final ClassLoader systemClassLoader, final File narWorkingDir) + throws InitializationException, IOException, ProcessorInstantiationException, NiFiRegistryException { + if (args == null) { + throw new IllegalArgumentException("Flow arguments can not be null"); + } + + System.out.println("Running flow from json: " + args.toString()); + + if (!args.has(REGISTRY) || !args.has(BUCKETID) || !args.has(FLOWID)) { + throw new IllegalArgumentException("The following parameters must be provided: " + REGISTRY + ", " + BUCKETID + ", " + FLOWID); + } + + final String registryurl = args.getAsJsonPrimitive(REGISTRY).getAsString(); + final String bucketID = args.getAsJsonPrimitive(BUCKETID).getAsString(); + final String flowID = args.getAsJsonPrimitive(FLOWID).getAsString(); + + int flowVersion = -1; + if (args.has(FLOWVERSION)) { + flowVersion = args.getAsJsonPrimitive(FLOWVERSION).getAsInt(); + } + + boolean materializeContent = true; + if (args.has(MATERIALIZECONTENT)) { + materializeContent = args.getAsJsonPrimitive(MATERIALIZECONTENT).getAsBoolean(); + } + + final List failurePorts = new ArrayList<>(); + if (args.has(FAILUREPORTS)) { + args.getAsJsonArray(FAILUREPORTS).forEach(port ->failurePorts.add(port.getAsString())); + } + + final Map inputVariables = new HashMap<>(); + + if (args.has(VARIABLES)) { + final JsonElement variablesElement = args.get(VARIABLES); + final JsonObject variablesObject = variablesElement.getAsJsonObject(); + variablesObject.entrySet() + .forEach(entry ->inputVariables.put(new VariableDescriptor(entry.getKey()), entry.getValue().getAsString())); + } + + final SSLContext sslContext = getSSLContext(args); + + final VersionedFlowSnapshot snapshot = new RegistryUtil(registryurl, sslContext).getFlowByID(bucketID, flowID, flowVersion); + final ExtensionManager extensionManager = ExtensionDiscovery.discover(narWorkingDir, systemClassLoader); + + final FnFlow flow = new FnFlow(snapshot.getFlowContents(), extensionManager, () -> inputVariables, failurePorts, materializeContent, sslContext); + flow.enqueueFromJSON(args); + return flow; + } + + public void enqueueFlowFile(final byte[] content, final Map attributes) { + if (sourceComponent == null) { + throw new IllegalArgumentException("Flow does not have an input port..."); + } + + //enqueue data + final Queue input = new LinkedList<>(); + input.add(new FnFlowFile(content, attributes, sourceComponent.isMaterializeContent())); + + sourceComponent.enqueueAll(input); + } + + public void enqueueFromJSON(final JsonObject json) { + final JsonArray flowFiles; + if (json.has(FLOWFILES)) { + flowFiles = json.getAsJsonArray(FLOWFILES); + } else { + return; + } + + if (flowFiles.size() == 0) { + return; + } + + if (sourceComponent == null) { + throw new IllegalStateException("Configuration specifies to inject " + flowFiles.size() + " FlowFiles into the flow, but the Flow does not contain an Input Port."); + } + + final Queue input = new LinkedList<>(); + flowFiles.forEach(f -> { + final JsonObject file = f.getAsJsonObject(); + final String content = file.getAsJsonPrimitive(CONTENT).getAsString(); + + final Map attributes = new HashMap<>(); + file.entrySet().forEach(entry -> { + if (!CONTENT.equals(entry.getKey())) { + attributes.put(entry.getKey(), entry.getValue().getAsString()); + } + }); + + input.add(new FnFlowFile(content, attributes, sourceComponent.isMaterializeContent())); + }); + + //enqueue data + sourceComponent.enqueueAll(input); + } +} diff --git a/nifi-fn/src/main/java/org/apache/nifi/fn/core/FnFlowFile.java b/nifi-fn/nifi-fn-core/src/main/java/org/apache/nifi/fn/core/FnFlowFile.java similarity index 88% rename from nifi-fn/src/main/java/org/apache/nifi/fn/core/FnFlowFile.java rename to nifi-fn/nifi-fn-core/src/main/java/org/apache/nifi/fn/core/FnFlowFile.java index 3c9012851c..cec98e2f9d 100644 --- a/nifi-fn/src/main/java/org/apache/nifi/fn/core/FnFlowFile.java +++ b/nifi-fn/nifi-fn-core/src/main/java/org/apache/nifi/fn/core/FnFlowFile.java @@ -17,22 +17,31 @@ package org.apache.nifi.fn.core; import com.google.gson.JsonObject; -import org.apache.commons.io.output.ByteArrayOutputStream; -import org.apache.nifi.controller.repository.FlowFileRecord; import org.apache.nifi.controller.repository.claim.ContentClaim; import org.apache.nifi.flowfile.FlowFile; import org.apache.nifi.flowfile.attributes.CoreAttributes; +import org.apache.nifi.fn.bootstrap.InMemoryFlowFile; import org.apache.nifi.processor.exception.FlowFileAccessException; +import org.apache.nifi.stream.io.StreamUtils; import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; import java.io.IOException; import java.io.InputStream; import java.io.SequenceInputStream; import java.nio.charset.StandardCharsets; -import java.util.*; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Set; +import java.util.UUID; import java.util.concurrent.atomic.AtomicLong; -public class FnFlowFile implements FlowFileRecord { +public class FnFlowFile implements InMemoryFlowFile { private final Map attributes = new HashMap<>(); private static AtomicLong nextID = new AtomicLong(0); @@ -50,10 +59,11 @@ public class FnFlowFile implements FlowFileRecord { private long lastEnqueuedDate = 0; private long enqueuedIndex = 0; - public FnFlowFile(String content, Map attributes, boolean materializeContent){ - this(content.getBytes(StandardCharsets.UTF_8), attributes,materializeContent); + public FnFlowFile(String content, Map attributes, boolean materializeContent) { + this(content.getBytes(StandardCharsets.UTF_8), attributes, materializeContent); } - public FnFlowFile(byte[] content, Map attributes, boolean materializeContent){ + + public FnFlowFile(byte[] content, Map attributes, boolean materializeContent) { this(materializeContent); this.attributes.putAll(attributes); @@ -70,9 +80,10 @@ public class FnFlowFile implements FlowFileRecord { try { this.setData(toCopy.getDataArray()); } catch (IOException e) { - throw new FlowFileAccessException("Exception creating FlowFile",e); + throw new FlowFileAccessException("Exception creating FlowFile", e); } } + public FnFlowFile(final FnFlowFile toCopy, long offset, long size, boolean materializeContent) { this(materializeContent); this.id = toCopy.id; @@ -83,9 +94,10 @@ public class FnFlowFile implements FlowFileRecord { try { this.setData(Arrays.copyOfRange(toCopy.getDataArray(), (int) offset, (int) (offset + size))); } catch (IOException e) { - throw new FlowFileAccessException("Exception creating FlowFile",e); + throw new FlowFileAccessException("Exception creating FlowFile", e); } } + public FnFlowFile(boolean materializeContent) { this.materializeContent = materializeContent; this.creationTime = System.nanoTime(); @@ -97,7 +109,6 @@ public class FnFlowFile implements FlowFileRecord { attributes.put(CoreAttributes.UUID.key(), UUID.randomUUID().toString()); } - //region SimpleMethods void setPenalized(boolean penalized) { this.penalized = penalized; @@ -210,7 +221,6 @@ public class FnFlowFile implements FlowFileRecord { } //endregion Methods - @Override public String toString() { JsonObject attributes = new JsonObject(); @@ -221,6 +231,7 @@ public class FnFlowFile implements FlowFileRecord { return result.toString(); } + public String toStringFull() { JsonObject attributes = new JsonObject(); this.attributes.forEach(attributes::addProperty); @@ -228,9 +239,9 @@ public class FnFlowFile implements FlowFileRecord { JsonObject result = new JsonObject(); result.add("attributes", attributes); try { - result.addProperty("content", new String(this.getDataArray(),StandardCharsets.UTF_8)); + result.addProperty("content", new String(this.getDataArray(), StandardCharsets.UTF_8)); } catch (IOException e) { - result.addProperty("content","Exception getting content: "+e.getMessage()); + result.addProperty("content", "Exception getting content: " + e.getMessage()); } return result.toString(); @@ -238,15 +249,15 @@ public class FnFlowFile implements FlowFileRecord { @Override public long getSize() { - if(isFullyMaterialized) + if (isFullyMaterialized) return data.length; else return 0; } public void addData(final byte[] data) { - if(materializeContent){ - ByteArrayOutputStream outputStream = new ByteArrayOutputStream( ); + if (materializeContent) { + ByteArrayOutputStream outputStream = new ByteArrayOutputStream(); try { outputStream.write(this.data); outputStream.write(data); @@ -260,12 +271,13 @@ public class FnFlowFile implements FlowFileRecord { this.dataStreams.add(new ByteArrayInputStream(data)); } } + public void addData(final InputStream in) { - if(materializeContent) { - ByteArrayOutputStream outputStream = new ByteArrayOutputStream( ); + if (materializeContent) { + ByteArrayOutputStream outputStream = new ByteArrayOutputStream(); try { outputStream.write(this.data); - outputStream.write(in); + StreamUtils.copy(in, outputStream); } catch (IOException e) { e.printStackTrace(); } @@ -281,11 +293,12 @@ public class FnFlowFile implements FlowFileRecord { this.data = data; isFullyMaterialized = true; } + public void setData(final InputStream in) { - if(materializeContent) { - ByteArrayOutputStream outputStream = new ByteArrayOutputStream( ); + if (materializeContent) { + ByteArrayOutputStream outputStream = new ByteArrayOutputStream(); try { - outputStream.write(in); + StreamUtils.copy(in, outputStream); } catch (IOException e) { e.printStackTrace(); } @@ -299,7 +312,7 @@ public class FnFlowFile implements FlowFileRecord { } public InputStream getDataStream() { - if(isFullyMaterialized) { + if (isFullyMaterialized) { return new ByteArrayInputStream(this.data); } else { return new SequenceInputStream( @@ -308,17 +321,19 @@ public class FnFlowFile implements FlowFileRecord { ); } } + public byte[] getDataArray() throws IOException { - if(!isFullyMaterialized) { + if (!isFullyMaterialized) { materializeData(); } return this.data; } + public void materializeData() throws IOException { InputStream in = this.getDataStream(); ByteArrayOutputStream baos = new ByteArrayOutputStream(); - byte[] buffer = new byte[1024*1024]; + byte[] buffer = new byte[1024 * 1024]; int read = 0; while ((read = in.read(buffer)) != -1) { baos.write(buffer, 0, read); diff --git a/nifi-fn/nifi-fn-core/src/main/java/org/apache/nifi/fn/core/FnPassThroughComponent.java b/nifi-fn/nifi-fn-core/src/main/java/org/apache/nifi/fn/core/FnPassThroughComponent.java new file mode 100644 index 0000000000..6e7c6f46f3 --- /dev/null +++ b/nifi-fn/nifi-fn-core/src/main/java/org/apache/nifi/fn/core/FnPassThroughComponent.java @@ -0,0 +1,75 @@ +/* + * 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.fn.core; + +import org.apache.nifi.fn.bootstrap.InMemoryFlowFile; +import org.apache.nifi.logging.ComponentLog; +import org.apache.nifi.processor.Relationship; + +import java.util.Collections; +import java.util.Queue; +import java.util.Set; + +public class FnPassThroughComponent extends AbstractFnComponent implements FnComponent { + static final Relationship RELATIONSHIP = new Relationship.Builder().name("").build(); + static final Set RELATIONSHIPS = Collections.singleton(RELATIONSHIP); + + private final FnConnectionContext connectionContext = new FnPassThroughConnectionContext(); + private final ComponentLog logger = new SLF4JComponentLog(this); + + + @Override + public Set getRelationships() { + return RELATIONSHIPS; + } + + @Override + protected FnConnectionContext getContext() { + return connectionContext; + } + + @Override + protected ComponentLog getLogger() { + return logger; + } + + @Override + public void shutdown() { + } + + @Override + public void enqueueAll(final Queue list) { + getChildren().get(RELATIONSHIP).forEach(child -> child.enqueueAll(list)); + } + + @Override + public boolean runRecursive(final Queue queue) { + for (final FnComponent component : getChildren().get(RELATIONSHIP)) { + final boolean success = component.runRecursive(queue); + if (!success) { + return false; + } + } + + return true; + } + + @Override + public boolean isMaterializeContent() { + return false; + } +} diff --git a/nifi-fn/nifi-fn-core/src/main/java/org/apache/nifi/fn/core/FnPassThroughConnectionContext.java b/nifi-fn/nifi-fn-core/src/main/java/org/apache/nifi/fn/core/FnPassThroughConnectionContext.java new file mode 100644 index 0000000000..890b298c03 --- /dev/null +++ b/nifi-fn/nifi-fn-core/src/main/java/org/apache/nifi/fn/core/FnPassThroughConnectionContext.java @@ -0,0 +1,36 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.fn.core; + +import org.apache.nifi.processor.Relationship; + +import java.util.HashSet; +import java.util.Set; + +public class FnPassThroughConnectionContext implements FnConnectionContext { + private final Set connectedRelationships = new HashSet<>(); + + @Override + public void addConnection(final Relationship relationship) { + connectedRelationships.add(relationship); + } + + @Override + public boolean isValid() { + return connectedRelationships.size() == 1 && connectedRelationships.contains(FnPassThroughComponent.RELATIONSHIP); + } +} diff --git a/nifi-fn/src/main/java/org/apache/nifi/fn/core/FnProcessContext.java b/nifi-fn/nifi-fn-core/src/main/java/org/apache/nifi/fn/core/FnProcessContext.java similarity index 87% rename from nifi-fn/src/main/java/org/apache/nifi/fn/core/FnProcessContext.java rename to nifi-fn/nifi-fn-core/src/main/java/org/apache/nifi/fn/core/FnProcessContext.java index 9dab31eaee..2ad2c86e78 100644 --- a/nifi-fn/src/main/java/org/apache/nifi/fn/core/FnProcessContext.java +++ b/nifi-fn/nifi-fn-core/src/main/java/org/apache/nifi/fn/core/FnProcessContext.java @@ -18,7 +18,11 @@ package org.apache.nifi.fn.core; import org.apache.nifi.annotation.behavior.InputRequirement; import org.apache.nifi.attribute.expression.language.Query; -import org.apache.nifi.components.*; +import org.apache.nifi.components.ConfigurableComponent; +import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.components.PropertyValue; +import org.apache.nifi.components.ValidationContext; +import org.apache.nifi.components.ValidationResult; import org.apache.nifi.components.state.StateManager; import org.apache.nifi.controller.ControllerService; import org.apache.nifi.controller.ControllerServiceInitializationContext; @@ -30,9 +34,18 @@ import org.apache.nifi.processor.SchedulingContext; import org.apache.nifi.registry.VariableRegistry; import java.io.File; -import java.util.*; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Set; -public class FnProcessContext implements SchedulingContext, ControllerServiceInitializationContext { +public class FnProcessContext implements SchedulingContext, ControllerServiceInitializationContext, FnConnectionContext { private final ConfigurableComponent component; private final String componentName; @@ -40,7 +53,6 @@ public class FnProcessContext implements SchedulingContext, ControllerServiceIni private final StateManager stateManager; private final VariableRegistry variableRegistry; - private String annotationData = null; private boolean yieldCalled = false; private boolean enableExpressionValidation = false; @@ -58,10 +70,13 @@ public class FnProcessContext implements SchedulingContext, ControllerServiceIni private final FnControllerServiceLookup lookup; - public FnProcessContext(final ConfigurableComponent component, final FnControllerServiceLookup lookup, final String componentName, final StateManager stateManager, final VariableRegistry variableRegistry) { + public FnProcessContext(final ConfigurableComponent component, final FnControllerServiceLookup lookup, final String componentName, final StateManager stateManager, final VariableRegistry + variableRegistry) { this(component, lookup, componentName, new SLF4JComponentLog(component), stateManager, variableRegistry); } - public FnProcessContext(final ConfigurableComponent component, final FnControllerServiceLookup lookup, final String componentName, final SLF4JComponentLog logger, final FnStateManager statemanager) { + + public FnProcessContext(final ConfigurableComponent component, final FnControllerServiceLookup lookup, final String componentName, final SLF4JComponentLog logger, final FnStateManager + statemanager) { this(component, lookup, componentName, logger, statemanager, VariableRegistry.EMPTY_REGISTRY); } @@ -77,12 +92,10 @@ public class FnProcessContext implements SchedulingContext, ControllerServiceIni this.lookup = lookup; this.stateManager = stateManager; this.variableRegistry = variableRegistry; - this.identifier = "ProcessContext-"+this.hashCode(); + this.identifier = "ProcessContext-" + this.hashCode(); this.logger = logger; } - - @Override public PropertyValue getProperty(final PropertyDescriptor descriptor) { return getProperty(descriptor.getName()); @@ -109,10 +122,11 @@ public class FnProcessContext implements SchedulingContext, ControllerServiceIni public ValidationResult setProperty(final String propertyName, final String propertyValue) { return setProperty(new PropertyDescriptor.Builder().name(propertyName).build(), propertyValue); } + public ValidationResult setProperty(final PropertyDescriptor descriptor, final String value) { - if(descriptor == null) + if (descriptor == null) throw new IllegalArgumentException("descriptor can not be null"); - if(value == null) + if (value == null) throw new IllegalArgumentException("Cannot set property to null value; if the intent is to remove the property, call removeProperty instead"); final PropertyDescriptor fullyPopulatedDescriptor = component.getPropertyDescriptor(descriptor.getName()); @@ -158,7 +172,6 @@ public class FnProcessContext implements SchedulingContext, ControllerServiceIni return yieldCalled; } - @Override public int getMaxConcurrentTasks() { return maxConcurrentTasks; @@ -190,7 +203,7 @@ public class FnProcessContext implements SchedulingContext, ControllerServiceIni @Override public Map getAllProperties() { - final Map propValueMap = new LinkedHashMap<>(); + final Map propValueMap = new LinkedHashMap<>(); for (final Map.Entry entry : getProperties().entrySet()) { propValueMap.put(entry.getKey().getName(), entry.getValue()); } @@ -210,9 +223,9 @@ public class FnProcessContext implements SchedulingContext, ControllerServiceIni for (Map.Entry service : this.lookup.getControllerServices().entrySet()) { if (!service.getValue().isEnabled()) { results.add(new ValidationResult.Builder() - .explanation("Controller service " + service.getKey() + " for " + this.getName() + " is not enabled") - .valid(false) - .build()); + .explanation("Controller service " + service.getKey() + " for " + this.getName() + " is not enabled") + .valid(false) + .build()); } } return results; @@ -239,11 +252,11 @@ public class FnProcessContext implements SchedulingContext, ControllerServiceIni final ControllerService controllerService = this.lookup.getControllerService(controllerServiceId); if (controllerService == null) { final ValidationResult result = new ValidationResult.Builder() - .valid(false) - .subject(descriptor.getDisplayName()) - .input(controllerServiceId) - .explanation("Invalid Controller Service: " + controllerServiceId + " is not a valid Controller Service Identifier") - .build(); + .valid(false) + .subject(descriptor.getDisplayName()) + .input(controllerServiceId) + .explanation("Invalid Controller Service: " + controllerServiceId + " is not a valid Controller Service Identifier") + .build(); validationResults.add(result); continue; @@ -252,11 +265,11 @@ public class FnProcessContext implements SchedulingContext, ControllerServiceIni final Class requiredServiceClass = descriptor.getControllerServiceDefinition(); if (!requiredServiceClass.isAssignableFrom(controllerService.getClass())) { final ValidationResult result = new ValidationResult.Builder() - .valid(false) - .subject(descriptor.getDisplayName()) - .input(controllerServiceId) - .explanation("Invalid Controller Service: " + controllerServiceId + " does not implement interface " + requiredServiceClass) - .build(); + .valid(false) + .subject(descriptor.getDisplayName()) + .input(controllerServiceId) + .explanation("Invalid Controller Service: " + controllerServiceId + " does not implement interface " + requiredServiceClass) + .build(); validationResults.add(result); continue; @@ -265,11 +278,11 @@ public class FnProcessContext implements SchedulingContext, ControllerServiceIni final boolean enabled = this.lookup.isControllerServiceEnabled(controllerServiceId); if (!enabled) { validationResults.add(new ValidationResult.Builder() - .input(controllerServiceId) - .subject(descriptor.getDisplayName()) - .explanation("Controller Service with ID " + controllerServiceId + " is not enabled") - .valid(false) - .build()); + .input(controllerServiceId) + .subject(descriptor.getDisplayName()) + .explanation("Controller Service with ID " + controllerServiceId + " is not enabled") + .valid(false) + .build()); } } @@ -301,7 +314,6 @@ public class FnProcessContext implements SchedulingContext, ControllerServiceIni return encrypted; } - @Override public ControllerServiceLookup getControllerServiceLookup() { return this.lookup; @@ -393,7 +405,6 @@ public class FnProcessContext implements SchedulingContext, ControllerServiceIni this.maxConcurrentTasks = maxConcurrentTasks; } - @Override public String getIdentifier() { return identifier; @@ -413,6 +424,7 @@ public class FnProcessContext implements SchedulingContext, ControllerServiceIni public File getKerberosServiceKeytab() { return null; //this needs to be wired in. } + @Override public File getKerberosConfigurationFile() { return null; //this needs to be wired in. diff --git a/nifi-fn/src/main/java/org/apache/nifi/fn/core/FnProcessSession.java b/nifi-fn/nifi-fn-core/src/main/java/org/apache/nifi/fn/core/FnProcessSession.java similarity index 90% rename from nifi-fn/src/main/java/org/apache/nifi/fn/core/FnProcessSession.java rename to nifi-fn/nifi-fn-core/src/main/java/org/apache/nifi/fn/core/FnProcessSession.java index 94b8021e3d..9c8b23550a 100644 --- a/nifi-fn/src/main/java/org/apache/nifi/fn/core/FnProcessSession.java +++ b/nifi-fn/nifi-fn-core/src/main/java/org/apache/nifi/fn/core/FnProcessSession.java @@ -32,12 +32,26 @@ import org.apache.nifi.processor.io.StreamCallback; import org.apache.nifi.provenance.ProvenanceEventRecord; import org.apache.nifi.provenance.ProvenanceReporter; -import java.io.*; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; import java.nio.file.Files; import java.nio.file.OpenOption; import java.nio.file.Path; import java.nio.file.StandardOpenOption; -import java.util.*; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Iterator; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Queue; +import java.util.Set; import java.util.concurrent.atomic.AtomicLong; import java.util.regex.Pattern; import java.util.stream.Collectors; @@ -63,16 +77,16 @@ public class FnProcessSession implements ProcessSession { private static final AtomicLong enqueuedIndex = new AtomicLong(0L); private final Runnable nextStep; //run before commit() completes - public FnProcessSession(final Queue input, Collection events, final Processor processor, Set outputRelationships, boolean materializeContent, Runnable nextStep) { + public FnProcessSession(final Queue input, final Collection events, final Processor processor, final Set outputRelationships, + final boolean materializeContent, final Runnable nextStep) { this.processor = processor; this.inputQueue = input; this.provenanceReporter = new ProvenanceCollector(this, events, processor.getIdentifier(), processor.getClass().getSimpleName()); this.materializeContent = materializeContent; this.nextStep = nextStep; - outputRelationships.forEach(r->outputMap.put(r,new LinkedList<>())); + outputRelationships.forEach(r -> outputMap.put(r, new LinkedList<>())); } - //region Attributes @Override @@ -84,7 +98,7 @@ public class FnProcessSession implements ProcessSession { if (!(flowFile instanceof FnFlowFile)) { throw new IllegalArgumentException("Cannot update attributes of a flow file that I did not create"); } - final FnFlowFile newFlowFile = new FnFlowFile((FnFlowFile)flowFile, this.materializeContent); + final FnFlowFile newFlowFile = new FnFlowFile((FnFlowFile) flowFile, this.materializeContent); currentVersions.put(newFlowFile.getId(), newFlowFile); newFlowFile.putAttributes(attrs); @@ -105,7 +119,7 @@ public class FnProcessSession implements ProcessSession { throw new IllegalArgumentException("Should not be attempting to set FlowFile UUID via putAttribute"); } - final FnFlowFile newFlowFile = new FnFlowFile( (FnFlowFile)flowFile, this.materializeContent); + final FnFlowFile newFlowFile = new FnFlowFile((FnFlowFile) flowFile, this.materializeContent); currentVersions.put(newFlowFile.getId(), newFlowFile); final Map attrs = new HashMap<>(); @@ -124,7 +138,7 @@ public class FnProcessSession implements ProcessSession { throw new IllegalArgumentException("Cannot export a flow file that I did not create"); } - final FnFlowFile newFlowFile = new FnFlowFile((FnFlowFile)flowFile, this.materializeContent); + final FnFlowFile newFlowFile = new FnFlowFile((FnFlowFile) flowFile, this.materializeContent); currentVersions.put(newFlowFile.getId(), newFlowFile); newFlowFile.removeAttributes(attrNames); @@ -160,7 +174,7 @@ public class FnProcessSession implements ProcessSession { if (!(flowFile instanceof FnFlowFile)) { throw new IllegalArgumentException("Cannot export a flow file that I did not create"); } - final FnFlowFile newFlowFile = new FnFlowFile((FnFlowFile)flowFile, this.materializeContent); + final FnFlowFile newFlowFile = new FnFlowFile((FnFlowFile) flowFile, this.materializeContent); currentVersions.put(newFlowFile.getId(), newFlowFile); final Set attrNames = new HashSet<>(); @@ -169,8 +183,6 @@ public class FnProcessSession implements ProcessSession { return newFlowFile; } - - /** * Inherits the attributes from the given source flow file into another flow * file. The UUID of the source becomes the parent UUID of this flow file. @@ -201,26 +213,6 @@ public class FnProcessSession implements ProcessSession { * @param sources to inherit common attributes from */ private FlowFile inheritAttributes(final Collection sources, final FlowFile destination) { - final StringBuilder parentUuidBuilder = new StringBuilder(); - int uuidsCaptured = 0; - for (final FlowFile source : sources) { - if (source == destination) { - continue; // don't want to capture parent uuid of this. Something can't be a child of itself - } - final String sourceUuid = source.getAttribute(CoreAttributes.UUID.key()); - if (sourceUuid != null && !sourceUuid.trim().isEmpty()) { - uuidsCaptured++; - if (parentUuidBuilder.length() > 0) { - parentUuidBuilder.append(","); - } - parentUuidBuilder.append(sourceUuid); - } - - if (uuidsCaptured > 100) { - break; - } - } - final FlowFile updated = putAllAttributes(destination, intersectAttributes(sources)); getProvenanceReporter().join(sources, updated); return updated; @@ -249,7 +241,8 @@ public class FnProcessSession implements ProcessSession { */ final Map firstMap = flowFileList.iterator().next().getAttributes(); - outer: for (final Map.Entry mapEntry : firstMap.entrySet()) { + outer: + for (final Map.Entry mapEntry : firstMap.entrySet()) { final String key = mapEntry.getKey(); final String value = mapEntry.getValue(); for (final FlowFile flowFile : flowFileList) { @@ -265,16 +258,14 @@ public class FnProcessSession implements ProcessSession { return result; } - //endregion - //region Metadata @Override @SuppressWarnings({"rawtypes", "unchecked"}) public void migrate(final ProcessSession newOwner, final Collection flowFiles) { Collection fnFlowFiles = (Collection) (Collection) flowFiles; - FnProcessSession newFnOwner = (FnProcessSession)newOwner; + FnProcessSession newFnOwner = (FnProcessSession) newOwner; if (Objects.requireNonNull(newOwner) == this) { throw new IllegalArgumentException("Cannot migrate FlowFiles from a Process Session to itself"); } @@ -327,8 +318,8 @@ public class FnProcessSession implements ProcessSession { } final Set flowFileIds = flowFiles.stream() - .map(ff -> ff.getAttribute(CoreAttributes.UUID.key())) - .collect(Collectors.toSet()); + .map(ff -> ff.getAttribute(CoreAttributes.UUID.key())) + .collect(Collectors.toSet()); provenanceReporter.migrate(newFnOwner.provenanceReporter, flowFileIds); } @@ -397,7 +388,7 @@ public class FnProcessSession implements ProcessSession { @Override public void rollback(final boolean penalize) { //if we've already committed then rollback is basically a no-op - if(committed){ + if (committed) { return; } @@ -441,13 +432,13 @@ public class FnProcessSession implements ProcessSession { // if the flowfile provided was created in this session (i.e. it's in currentVersions and not in original versions), // then throw an exception indicating that you can't transfer flowfiles back to self. // this mimics the same behavior in StandardProcessSession - if(currentVersions.get(flowFile.getId()) != null && originalVersions.get(flowFile.getId()) == null) { + if (currentVersions.get(flowFile.getId()) != null && originalVersions.get(flowFile.getId()) == null) { throw new IllegalArgumentException("Cannot transfer FlowFiles that are created in this Session back to self"); } beingProcessed.remove(flowFile.getId()); - inputQueue.add((FnFlowFile)flowFile); - updateLastQueuedDate((FnFlowFile)flowFile); + inputQueue.add((FnFlowFile) flowFile); + updateLastQueuedDate((FnFlowFile) flowFile); } @@ -462,13 +453,13 @@ public class FnProcessSession implements ProcessSession { transfer(flowFile); return; } - if(!processor.getRelationships().contains(relationship)){ + if (!processor.getRelationships().contains(relationship)) { throw new IllegalArgumentException("this relationship " + relationship.getName() + " is not known"); } flowFile = validateState(flowFile); - if(outputMap.containsKey(relationship)){ + if (outputMap.containsKey(relationship)) { Queue queue = this.outputMap.get(relationship); queue.add((FnFlowFile) flowFile); @@ -492,10 +483,11 @@ public class FnProcessSession implements ProcessSession { public ProvenanceReporter getProvenanceReporter() { return provenanceReporter; } + @Override public FnFlowFile penalize(FlowFile flowFile) { flowFile = validateState(flowFile); - final FnFlowFile newFlowFile = new FnFlowFile((FnFlowFile)flowFile, this.materializeContent); + final FnFlowFile newFlowFile = new FnFlowFile((FnFlowFile) flowFile, this.materializeContent); currentVersions.put(newFlowFile.getId(), newFlowFile); newFlowFile.setPenalized(true); penalized.add(newFlowFile); @@ -599,7 +591,6 @@ public class FnProcessSession implements ProcessSession { //endregion - @Override public void commit() { if (!beingProcessed.isEmpty()) { @@ -618,7 +609,7 @@ public class FnProcessSession implements ProcessSession { @Override public FnFlowFile clone(FlowFile flowFile) { flowFile = validateState(flowFile); - final FnFlowFile newFlowFile = new FnFlowFile((FnFlowFile)flowFile, this.materializeContent); + final FnFlowFile newFlowFile = new FnFlowFile((FnFlowFile) flowFile, this.materializeContent); currentVersions.put(newFlowFile.getId(), newFlowFile); beingProcessed.add(newFlowFile.getId()); return newFlowFile; @@ -628,18 +619,16 @@ public class FnProcessSession implements ProcessSession { public FnFlowFile clone(FlowFile flowFile, final long offset, final long size) { flowFile = validateState(flowFile); try { - ((FnFlowFile)flowFile).materializeData(); + ((FnFlowFile) flowFile).materializeData(); } catch (IOException e) { - e.printStackTrace(); - throw new FlowFileHandlingException("Error materializing data",e); + throw new FlowFileHandlingException("Error materializing data", e); } if (offset + size > flowFile.getSize()) { throw new FlowFileHandlingException("Specified offset of " + offset + " and size " + size + " exceeds size of " + flowFile.toString()); } - final FnFlowFile newFlowFile = new FnFlowFile((FnFlowFile)flowFile, offset, size, this.materializeContent); - + final FnFlowFile newFlowFile = new FnFlowFile((FnFlowFile) flowFile, offset, size, this.materializeContent); currentVersions.put(newFlowFile.getId(), newFlowFile); beingProcessed.add(newFlowFile.getId()); @@ -658,7 +647,7 @@ public class FnProcessSession implements ProcessSession { } try { - copyTo(((FnFlowFile) flowFile).getDataStream(),out); + copyTo(((FnFlowFile) flowFile).getDataStream(), out); } catch (final IOException e) { throw new FlowFileAccessException(e.toString(), e); } @@ -673,14 +662,14 @@ public class FnProcessSession implements ProcessSession { if (!(flowFile instanceof FnFlowFile)) { throw new IllegalArgumentException("Cannot export a flow file that I did not create"); } - FnFlowFile fnFlowFile = (FnFlowFile)flowFile; + FnFlowFile fnFlowFile = (FnFlowFile) flowFile; final OpenOption mode = append ? StandardOpenOption.APPEND : StandardOpenOption.CREATE; try (final OutputStream out = Files.newOutputStream(path, mode)) { - if(fnFlowFile.materializeContent) + if (fnFlowFile.materializeContent) fnFlowFile.materializeData(); - copyTo(fnFlowFile.getDataStream(),out); + copyTo(fnFlowFile.getDataStream(), out); } catch (final IOException e) { throw new FlowFileAccessException(e.toString(), e); } @@ -696,7 +685,7 @@ public class FnProcessSession implements ProcessSession { throw new IllegalArgumentException("Cannot export a flow file that I did not create"); } - final FnFlowFile newFlowFile = new FnFlowFile((FnFlowFile)flowFile, this.materializeContent); + final FnFlowFile newFlowFile = new FnFlowFile((FnFlowFile) flowFile, this.materializeContent); newFlowFile.setData(in); currentVersions.put(newFlowFile.getId(), newFlowFile); @@ -713,10 +702,10 @@ public class FnProcessSession implements ProcessSession { if (!(flowFile instanceof FnFlowFile)) { throw new IllegalArgumentException("Cannot export a flow file that I did not create"); } - if(keepSourceFile == false){ + if (!keepSourceFile) { throw new IllegalArgumentException("Not going to delete the file..."); } - FnFlowFile newFlowFile = new FnFlowFile((FnFlowFile)flowFile, this.materializeContent); + FnFlowFile newFlowFile = new FnFlowFile((FnFlowFile) flowFile, this.materializeContent); try { newFlowFile.setData(Files.newInputStream(path)); } catch (IOException e) { @@ -743,10 +732,11 @@ public class FnProcessSession implements ProcessSession { if (!(flowFile instanceof FnFlowFile)) { throw new IllegalArgumentException("Cannot export a flow file that I did not create"); } + //allowSessionStreamManagement not used... try { - ((FnFlowFile)flowFile).materializeData(); - callback.process(((FnFlowFile)flowFile).getDataStream()); + ((FnFlowFile) flowFile).materializeData(); + callback.process(((FnFlowFile) flowFile).getDataStream()); } catch (final IOException e) { throw new ProcessException(e.toString(), e); } @@ -756,7 +746,7 @@ public class FnProcessSession implements ProcessSession { public InputStream read(FlowFile flowFile) { flowFile = validateState(flowFile); - return ((FnFlowFile)flowFile).getDataStream(); + return ((FnFlowFile) flowFile).getDataStream(); } @Override @@ -775,7 +765,7 @@ public class FnProcessSession implements ProcessSession { throw new ProcessException(e.toString(), e); } - final FnFlowFile newFlowFile = new FnFlowFile((FnFlowFile)flowFile, this.materializeContent); + final FnFlowFile newFlowFile = new FnFlowFile((FnFlowFile) flowFile, this.materializeContent); newFlowFile.setData(baos.toByteArray()); currentVersions.put(newFlowFile.getId(), newFlowFile); return newFlowFile; @@ -792,7 +782,7 @@ public class FnProcessSession implements ProcessSession { @Override public void close() throws IOException { super.close(); - final FnFlowFile newFlowFile = new FnFlowFile((FnFlowFile)flowFile, materializeContent); + final FnFlowFile newFlowFile = new FnFlowFile((FnFlowFile) flowFile, materializeContent); currentVersions.put(newFlowFile.getId(), newFlowFile); } }; @@ -847,9 +837,9 @@ public class FnProcessSession implements ProcessSession { public FnFlowFile merge(Collection sources, FlowFile destination) { sources = validateState(sources); destination = validateState(destination); - final FnFlowFile newFlowFile = new FnFlowFile((FnFlowFile)destination, this.materializeContent); + final FnFlowFile newFlowFile = new FnFlowFile((FnFlowFile) destination, this.materializeContent); for (final FlowFile flowFile : sources) { - newFlowFile.addData(((FnFlowFile)flowFile).getDataStream()); + newFlowFile.addData(((FnFlowFile) flowFile).getDataStream()); } currentVersions.put(newFlowFile.getId(), newFlowFile); @@ -858,7 +848,7 @@ public class FnProcessSession implements ProcessSession { @Override public FnFlowFile merge(Collection sources, FlowFile destination, byte[] header, byte[] footer, byte[] demarcator) { - Collection fnSources = (Collection)validateState(sources); + Collection fnSources = (Collection) validateState(sources); FnFlowFile fnDestination = validateState(destination); if (header != null) { @@ -885,7 +875,7 @@ public class FnProcessSession implements ProcessSession { public FnFlowFile unpenalize(FlowFile flowFile) { flowFile = validateState(flowFile); - final FnFlowFile newFlowFile = new FnFlowFile((FnFlowFile)flowFile, this.materializeContent); + final FnFlowFile newFlowFile = new FnFlowFile((FnFlowFile) flowFile, this.materializeContent); currentVersions.put(newFlowFile.getId(), newFlowFile); newFlowFile.setPenalized(false); penalized.remove(newFlowFile); @@ -893,10 +883,8 @@ public class FnProcessSession implements ProcessSession { } //endregion - //region Utility - boolean isFlowFileKnown(final FlowFile flowFile) { final FlowFile curFlowFile = currentVersions.get(flowFile.getId()); if (curFlowFile == null) { @@ -912,11 +900,10 @@ public class FnProcessSession implements ProcessSession { return true; } - private List validateState(final Collection flowFiles) { return flowFiles.stream() - .map(ff -> validateState(ff)) - .collect(Collectors.toList()); + .map(this::validateState) + .collect(Collectors.toList()); } private FnFlowFile validateState(final FlowFile flowFile) { @@ -936,20 +923,21 @@ public class FnProcessSession implements ProcessSession { return currentVersion; } - public boolean isCommitted(){ + public boolean isCommitted() { return committed; } - public boolean isRolledback(){ + + public boolean isRolledback() { return rolledback; } - public boolean isInputQueueEmpty(){ + public boolean isInputQueueEmpty() { return this.inputQueue.isEmpty(); } - public boolean areAllFlowFilesTransfered(final Relationship relationship){ - if(outputMap.containsKey(relationship)) { - if(!outputMap.get(relationship).isEmpty()) + public boolean areAllFlowFilesTransfered(final Relationship relationship) { + if (outputMap.containsKey(relationship)) { + if (!outputMap.get(relationship).isEmpty()) return false; } return true; @@ -963,11 +951,11 @@ public class FnProcessSession implements ProcessSession { return removedFlowFiles.size(); } - public Queue getAndRemoveFlowFilesForRelationship(final String relationship) { final Relationship procRel = new Relationship.Builder().name(relationship).build(); return getAndRemoveFlowFilesForRelationship(procRel); } + public Queue getAndRemoveFlowFilesForRelationship(final Relationship relationship) { Queue queue = this.outputMap.get(relationship); if (queue == null) { @@ -982,7 +970,6 @@ public class FnProcessSession implements ProcessSession { return penalized; } - private void updateLastQueuedDate(FnFlowFile FnFlowFile) { // Simulate StandardProcessSession.updateLastQueuedDate, // which is called when a flow file is transferred to a relationship. @@ -990,9 +977,8 @@ public class FnProcessSession implements ProcessSession { FnFlowFile.setEnqueuedIndex(enqueuedIndex.incrementAndGet()); } - private void copyTo(InputStream in, OutputStream out) throws IOException { - byte[] buffer = new byte[1024*1024]; + byte[] buffer = new byte[1024 * 1024]; int len; while ((len = in.read(buffer)) != -1) { out.write(buffer, 0, len); diff --git a/nifi-fn/src/main/java/org/apache/nifi/fn/core/FnProcessorInitializationContext.java b/nifi-fn/nifi-fn-core/src/main/java/org/apache/nifi/fn/core/FnProcessorInitializationContext.java similarity index 78% rename from nifi-fn/src/main/java/org/apache/nifi/fn/core/FnProcessorInitializationContext.java rename to nifi-fn/nifi-fn-core/src/main/java/org/apache/nifi/fn/core/FnProcessorInitializationContext.java index ee41e68370..a9bd8c2a02 100644 --- a/nifi-fn/src/main/java/org/apache/nifi/fn/core/FnProcessorInitializationContext.java +++ b/nifi-fn/nifi-fn-core/src/main/java/org/apache/nifi/fn/core/FnProcessorInitializationContext.java @@ -24,17 +24,22 @@ import org.apache.nifi.processor.Processor; import org.apache.nifi.processor.ProcessorInitializationContext; import java.io.File; -import java.util.UUID; public class FnProcessorInitializationContext implements ProcessorInitializationContext { private final ComponentLog logger; private final String processorId; - private final ProcessContext context; + private final ControllerServiceLookup controllerServiceLookup; - public FnProcessorInitializationContext(final Processor processor, final ProcessContext context) { - processorId = UUID.randomUUID().toString(); + public FnProcessorInitializationContext(final String id, final Processor processor, final ProcessContext context) { + processorId = id; logger = new SLF4JComponentLog(processor); - this.context = context; + this.controllerServiceLookup = context.getControllerServiceLookup(); + } + + public FnProcessorInitializationContext(final String id, final Processor processor, final ControllerServiceLookup controllerServiceLookup) { + processorId = id; + logger = new SLF4JComponentLog(processor); + this.controllerServiceLookup = controllerServiceLookup; } public String getIdentifier() { @@ -46,7 +51,7 @@ public class FnProcessorInitializationContext implements ProcessorInitialization } public ControllerServiceLookup getControllerServiceLookup() { - return context.getControllerServiceLookup(); + return controllerServiceLookup; } public NodeTypeProvider getNodeTypeProvider() { diff --git a/nifi-fn/nifi-fn-core/src/main/java/org/apache/nifi/fn/core/FnProcessorWrapper.java b/nifi-fn/nifi-fn-core/src/main/java/org/apache/nifi/fn/core/FnProcessorWrapper.java new file mode 100644 index 0000000000..45d60e3f77 --- /dev/null +++ b/nifi-fn/nifi-fn-core/src/main/java/org/apache/nifi/fn/core/FnProcessorWrapper.java @@ -0,0 +1,332 @@ +/* + * 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.fn.core; + +import org.apache.nifi.annotation.lifecycle.OnAdded; +import org.apache.nifi.annotation.lifecycle.OnConfigurationRestored; +import org.apache.nifi.annotation.lifecycle.OnScheduled; +import org.apache.nifi.annotation.lifecycle.OnShutdown; +import org.apache.nifi.annotation.lifecycle.OnStopped; +import org.apache.nifi.annotation.lifecycle.OnUnscheduled; +import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.components.ValidationResult; +import org.apache.nifi.fn.bootstrap.InMemoryFlowFile; +import org.apache.nifi.logging.ComponentLog; +import org.apache.nifi.processor.Processor; +import org.apache.nifi.processor.Relationship; +import org.apache.nifi.provenance.ProvenanceEventRecord; +import org.apache.nifi.registry.VariableRegistry; + +import java.io.Closeable; +import java.lang.reflect.InvocationTargetException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Comparator; +import java.util.HashSet; +import java.util.LinkedList; +import java.util.List; +import java.util.Queue; +import java.util.Set; +import java.util.concurrent.CopyOnWriteArraySet; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.stream.Collectors; + +public class FnProcessorWrapper extends AbstractFnComponent implements FnComponent { + + private final boolean materializeContent; + private final Processor processor; + private final FnProcessContext context; + private final Queue inputQueue; + private final VariableRegistry variableRegistry; + private final ClassLoader classLoader; + + private final Collection provenanceEvents; + + private final Set createdSessions; + private final ComponentLog logger; + + private final FnControllerServiceLookup lookup; + + private volatile boolean stopRequested = false; + private volatile boolean isStopped = true; + private volatile boolean initialized = false; + + + FnProcessorWrapper(final String id, final Processor processor, final FnProcessorWrapper parent, final FnControllerServiceLookup lookup, final VariableRegistry registry, + final boolean materializeContent, final ClassLoader classLoader) throws InvocationTargetException, IllegalAccessException { + + this.processor = processor; + this.classLoader = classLoader; + + addParent(parent); + + this.lookup = lookup; + this.materializeContent = materializeContent; + + this.provenanceEvents = new ArrayList<>(); + this.createdSessions = new CopyOnWriteArraySet<>(); + this.inputQueue = new LinkedList<>(); + this.variableRegistry = registry; + this.context = new FnProcessContext(processor, lookup, processor.getIdentifier(), new FnStateManager(), variableRegistry); + this.context.setMaxConcurrentTasks(1); + + final FnProcessorInitializationContext initContext = new FnProcessorInitializationContext(id, processor, context); + logger = initContext.getLogger(); + + try (final CloseableNarLoader c = withNarClassLoader()) { + processor.initialize(initContext); + ReflectionUtils.invokeMethodsWithAnnotation(OnAdded.class, processor); + ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnConfigurationRestored.class, processor); + } + } + + private Processor getProcessor() { + return this.processor; + } + + public Set getRelationships() { + return processor.getRelationships(); + } + + + private void initialize() { + //Validate context + final Collection validationResult = context.validate(); + if (validationResult.stream().anyMatch(a -> !a.isValid()) || !this.validate()) { + throw new IllegalArgumentException("Processor is not valid: " + + String.join("\n", validationResult.stream().map(ValidationResult::toString).collect(Collectors.toList()))); + } + + try (final CloseableNarLoader c = withNarClassLoader()) { + ReflectionUtils.invokeMethodsWithAnnotation(OnScheduled.class, processor, context); + } catch (final Exception e) { + logger.error("Failed to perform @OnScheduled Lifecycle method: ", e); + } + + initialized = true; + } + + private CloseableNarLoader withNarClassLoader() { + final ClassLoader contextclassLoader = Thread.currentThread().getContextClassLoader(); + Thread.currentThread().setContextClassLoader(this.classLoader); + + return new CloseableNarLoader() { + @Override + public void close() { + Thread.currentThread().setContextClassLoader(contextclassLoader); + } + }; + } + + public boolean runRecursive(final Queue output) { + if (!initialized) { + initialize(); + } + + final AtomicBoolean processingSuccess = new AtomicBoolean(true); + final Set outputRelationships = new HashSet<>(getChildren().keySet()); + outputRelationships.addAll(getSuccessOutputPorts()); + outputRelationships.addAll(getFailureOutputPorts()); + + do { + this.isStopped = false; + final AtomicBoolean nextStepCalled = new AtomicBoolean(false); + + try { + logger.info("Running " + this.processor.getClass().getSimpleName() + ".onTrigger with " + inputQueue.size() + " FlowFiles"); + + try (final CloseableNarLoader c = withNarClassLoader()) { // Trigger processor with the appropriate class loader + processor.onTrigger(context, () -> { + final FnProcessSession session = new FnProcessSession( + inputQueue, + provenanceEvents, + processor, + outputRelationships, + materializeContent, + () -> { + if (!nextStepCalled.get()) { + nextStepCalled.set(true); + boolean successfulRun = runChildren(output); + processingSuccess.set(successfulRun); + } + }); + + createdSessions.add(session); + return session; + }); + } + + if (!nextStepCalled.get()) { + nextStepCalled.set(true); + boolean successfulRun = runChildren(output); + processingSuccess.set(successfulRun); + } + + provenanceEvents.clear(); + } catch (final Exception t) { + try (final CloseableNarLoader c = withNarClassLoader()) { + logger.error("Failed to trigger " + this.processor, t); + } + + return false; + } + } while (!stopRequested && !inputQueue.isEmpty() && processingSuccess.get()); + + this.isStopped = true; + return processingSuccess.get(); + } + + private boolean runChildren(final Queue output) { + final Queue penalizedFlowFiles = this.getPenalizedFlowFiles(); + if (penalizedFlowFiles.size() > 0) { + output.addAll(penalizedFlowFiles); + return false; + } + + for (final Relationship relationship : this.getProcessor().getRelationships()) { + if (isAutoTerminated(relationship)) { + continue; + } + + final Queue files = this.getAndRemoveFlowFilesForRelationship(relationship); + if (files.size() == 0) { + continue; + } + + if (getFailureOutputPorts().contains(relationship)) { + output.addAll(files); + return false; + } + + if (getSuccessOutputPorts().contains(relationship)) { + output.addAll(files); + } + + final List childComponents = getChildren().get(relationship); + if (childComponents != null) { + for (final FnComponent child : childComponents) { + child.enqueueAll(files); + boolean successfulRun = child.runRecursive(output); + + if (!successfulRun) { + return false; + } + } + } + } + + return true; + } + + public void shutdown() { + this.stopRequested = true; + + for (Relationship relationship : this.getProcessor().getRelationships()) { + if (isAutoTerminated(relationship)) { + continue; + } + + final List childComponents = getChildren().get(relationship); + if (childComponents == null) { + throw new IllegalArgumentException("No child for relationship: " + relationship.getName()); + } + + childComponents.forEach(FnComponent::shutdown); + } + + while (!this.isStopped) { + try { + Thread.sleep(10); + } catch (InterruptedException e) { + e.printStackTrace(); + break; + } + } + + try (final CloseableNarLoader c = withNarClassLoader()) { + ReflectionUtils.invokeMethodsWithAnnotation(OnUnscheduled.class, processor, context); + + ReflectionUtils.invokeMethodsWithAnnotation(OnStopped.class, processor, context); + + ReflectionUtils.invokeMethodsWithAnnotation(OnShutdown.class, processor); + } catch (final Exception e) { + logger.error("Failed to properly shutdown " + processor + ": ", e); + } + + logger.info(this.processor.getClass().getSimpleName() + " shutdown"); + } + + + public void enqueueAll(final Queue list) { + inputQueue.addAll(list); + } + + public Queue getAndRemoveFlowFilesForRelationship(final Relationship relationship) { + final Queue sortedList = createdSessions.stream() + .flatMap(s -> s.getAndRemoveFlowFilesForRelationship(relationship).stream()) + .sorted(Comparator.comparing(FnFlowFile::getCreationTime)) + .collect(Collectors.toCollection(LinkedList::new)); + + return sortedList; + } + + public Queue getPenalizedFlowFiles() { + final Queue sortedList = createdSessions.stream() + .flatMap(s -> s.getPenalizedFlowFiles().stream()) + .sorted(Comparator.comparing(FnFlowFile::getCreationTime)) + .collect(Collectors.toCollection(LinkedList::new)); + return sortedList; + + } + + public ValidationResult setProperty(final PropertyDescriptor property, final String propertyValue) { + return context.setProperty(property, propertyValue); + } + + public ValidationResult setProperty(final String propertyName, final String propertyValue) { + return context.setProperty(propertyName, propertyValue); + } + + public void setAnnotationData(final String annotationData) { + context.setAnnotationData(annotationData); + } + + public boolean isMaterializeContent() { + return materializeContent; + } + + @Override + public ComponentLog getLogger() { + return logger; + } + + @Override + protected FnProcessContext getContext() { + return context; + } + + /** + * A simple interface that extends Closeable in order to provide a close() method that does not throw any checked + * Exceptions. This is done so that the {@link #withNarClassLoader()} is able to be used without having to catch + * an IOException that will never be thrown. + */ + private interface CloseableNarLoader extends Closeable { + @Override + void close(); + } +} + diff --git a/nifi-fn/src/main/java/org/apache/nifi/fn/core/FnPropertyValue.java b/nifi-fn/nifi-fn-core/src/main/java/org/apache/nifi/fn/core/FnPropertyValue.java similarity index 78% rename from nifi-fn/src/main/java/org/apache/nifi/fn/core/FnPropertyValue.java rename to nifi-fn/nifi-fn-core/src/main/java/org/apache/nifi/fn/core/FnPropertyValue.java index d8528f27a4..16625f6c5e 100644 --- a/nifi-fn/src/main/java/org/apache/nifi/fn/core/FnPropertyValue.java +++ b/nifi-fn/nifi-fn-core/src/main/java/org/apache/nifi/fn/core/FnPropertyValue.java @@ -24,11 +24,11 @@ import org.apache.nifi.components.PropertyValue; import org.apache.nifi.controller.ControllerService; import org.apache.nifi.controller.ControllerServiceLookup; import org.apache.nifi.expression.AttributeValueDecorator; +import org.apache.nifi.expression.ExpressionLanguageScope; import org.apache.nifi.flowfile.FlowFile; import org.apache.nifi.processor.DataUnit; import org.apache.nifi.processor.exception.ProcessException; import org.apache.nifi.registry.VariableRegistry; -import org.apache.nifi.expression.ExpressionLanguageScope; import java.util.HashMap; import java.util.List; @@ -74,62 +74,42 @@ public class FnPropertyValue implements PropertyValue { this.variableRegistry = variableRegistry; } - private void ensureExpressionsEvaluated() { if (Boolean.TRUE.equals(expectExpressions) && !expressionsEvaluated) { throw new IllegalStateException("Attempting to retrieve value of " + propertyDescriptor - + " without first evaluating Expressions, even though the PropertyDescriptor indicates " - + "that the Expression Language is Supported. If you realize that this is the case and do not want " - + "this error to occur, it can be disabled by calling TestRunner.setValidateExpressionUsage(false)"); + + " without first evaluating Expressions, even though the PropertyDescriptor indicates " + + "that the Expression Language is Supported. If you realize that this is the case and do not want " + + "this error to occur, it can be disabled by calling TestRunner.setValidateExpressionUsage(false)"); } } private void validateExpressionScope(boolean attributesAvailable) { // language scope is not null, we have attributes available but scope is not equal to FF attributes // it means that we're not evaluating against flow file attributes even though attributes are available - if(expressionLanguageScope != null - && (attributesAvailable && !ExpressionLanguageScope.FLOWFILE_ATTRIBUTES.equals(expressionLanguageScope))) { + if (expressionLanguageScope != null + && (attributesAvailable && !ExpressionLanguageScope.FLOWFILE_ATTRIBUTES.equals(expressionLanguageScope))) { throw new IllegalStateException("Attempting to evaluate expression language for " + propertyDescriptor.getName() - + " using flow file attributes but the scope evaluation is set to " + expressionLanguageScope + ". The" - + " proper scope should be set in the property descriptor using" - + " PropertyDescriptor.Builder.expressionLanguageSupported(ExpressionLanguageScope)"); + + " using flow file attributes but the scope evaluation is set to " + expressionLanguageScope + ". The" + + " proper scope should be set in the property descriptor using" + + " PropertyDescriptor.Builder.expressionLanguageSupported(ExpressionLanguageScope)"); } // if the service lookup is an instance of the validation context, we're in the validate() method // at this point we don't have any flow file available and we should not care about the scope // even though it is defined as FLOWFILE_ATTRIBUTES - if(expressionLanguageScope != null - && ExpressionLanguageScope.FLOWFILE_ATTRIBUTES.equals(expressionLanguageScope) - && this.serviceLookup instanceof FnControllerServiceLookup) { + if (expressionLanguageScope != null + && ExpressionLanguageScope.FLOWFILE_ATTRIBUTES.equals(expressionLanguageScope) + && this.serviceLookup instanceof FnControllerServiceLookup) { return; } - // we check if the input requirement is INPUT_FORBIDDEN - // in that case, we don't care if attributes are not available even though scope is FLOWFILE_ATTRIBUTES - // it likely means that the property has been defined in a common/abstract class used by multiple processors with - // different input requirements. - //if(expressionLanguageScope != null - // && ExpressionLanguageScope.FLOWFILE_ATTRIBUTES.equals(expressionLanguageScope) - // && (this.serviceLookup.getInputRequirement() == null - // || this.serviceLookup.getInputRequirement().value().equals(InputRequirement.Requirement.INPUT_FORBIDDEN))) { - // return; - //} - - // if we have a processor where input requirement is INPUT_ALLOWED, we need to check if there is an - // incoming connection or not. If not, we don't care if attributes are not available even though scope is FLOWFILE_ATTRIBUTES - //if(expressionLanguageScope != null - // && ExpressionLanguageScope.FLOWFILE_ATTRIBUTES.equals(expressionLanguageScope) - // && !((org.apache.nifi.fn.core.FnProcessContext) this.context).hasIncomingConnection()) { - // return; - //} - // we're trying to evaluate against flow files attributes but we don't have any attributes available. - if(expressionLanguageScope != null - && (!attributesAvailable && ExpressionLanguageScope.FLOWFILE_ATTRIBUTES.equals(expressionLanguageScope))) { + if (expressionLanguageScope != null + && (!attributesAvailable && ExpressionLanguageScope.FLOWFILE_ATTRIBUTES.equals(expressionLanguageScope))) { throw new IllegalStateException("Attempting to evaluate expression language for " + propertyDescriptor.getName() - + " without using flow file attributes but the scope evaluation is set to " + expressionLanguageScope + ". The" - + " proper scope should be set in the property descriptor using" - + " PropertyDescriptor.Builder.expressionLanguageSupported(ExpressionLanguageScope)"); + + " without using flow file attributes but the scope evaluation is set to " + expressionLanguageScope + ". The" + + " proper scope should be set in the property descriptor using" + + " PropertyDescriptor.Builder.expressionLanguageSupported(ExpressionLanguageScope)"); } } @@ -184,8 +164,8 @@ public class FnPropertyValue implements PropertyValue { private void markEvaluated() { if (Boolean.FALSE.equals(expectExpressions)) { throw new IllegalStateException("Attempting to Evaluate Expressions but " + propertyDescriptor - + " indicates that the Expression Language is not supported. If you realize that this is the case and do not want " - + "this error to occur, it can be disabled by calling TestRunner.setValidateExpressionUsage(false)"); + + " indicates that the Expression Language is not supported. If you realize that this is the case and do not want " + + "this error to occur, it can be disabled by calling TestRunner.setValidateExpressionUsage(false)"); } expressionsEvaluated = true; } @@ -245,7 +225,7 @@ public class FnPropertyValue implements PropertyValue { @Override public PropertyValue evaluateAttributeExpressions(FlowFile flowFile, Map additionalAttributes, AttributeValueDecorator decorator, Map stateValues) - throws ProcessException { + throws ProcessException { markEvaluated(); if (rawValue == null) { return this; diff --git a/nifi-fn/nifi-fn-core/src/main/java/org/apache/nifi/fn/core/FnRemoteInputPort.java b/nifi-fn/nifi-fn-core/src/main/java/org/apache/nifi/fn/core/FnRemoteInputPort.java new file mode 100644 index 0000000000..e608384ec7 --- /dev/null +++ b/nifi-fn/nifi-fn-core/src/main/java/org/apache/nifi/fn/core/FnRemoteInputPort.java @@ -0,0 +1,130 @@ +/* + * 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.fn.core; + +import org.apache.nifi.events.EventReporter; +import org.apache.nifi.fn.bootstrap.InMemoryFlowFile; +import org.apache.nifi.logging.ComponentLog; +import org.apache.nifi.processor.Relationship; +import org.apache.nifi.registry.flow.VersionedRemoteGroupPort; +import org.apache.nifi.registry.flow.VersionedRemoteProcessGroup; +import org.apache.nifi.remote.Transaction; +import org.apache.nifi.remote.TransferDirection; +import org.apache.nifi.remote.client.SiteToSiteClient; +import org.apache.nifi.remote.protocol.DataPacket; +import org.apache.nifi.remote.protocol.SiteToSiteTransportProtocol; +import org.apache.nifi.remote.util.StandardDataPacket; +import org.apache.nifi.util.FormatUtils; + +import javax.net.ssl.SSLContext; +import java.util.Collections; +import java.util.LinkedList; +import java.util.Queue; +import java.util.Set; +import java.util.concurrent.TimeUnit; + +public class FnRemoteInputPort extends AbstractFnComponent { + private final Queue inputQueue = new LinkedList<>(); + private final SiteToSiteClient client; + private final String name; + private final String url; + + private final ComponentLog logger = new SLF4JComponentLog(this); + private final FnConnectionContext connectionContext = new FnConnectionContext() { + @Override + public void addConnection(final Relationship relationship) { + } + + @Override + public boolean isValid() { + return true; + } + }; + + public FnRemoteInputPort(final VersionedRemoteProcessGroup rpg, final VersionedRemoteGroupPort remotePort, final SSLContext sslContext) { + final String timeout = rpg.getCommunicationsTimeout(); + final long timeoutMillis = FormatUtils.getTimeDuration(timeout, TimeUnit.MILLISECONDS); + + url = rpg.getTargetUris(); + name = remotePort.getName(); + + client = new SiteToSiteClient.Builder() + .portName(remotePort.getName()) + .timeout(timeoutMillis, TimeUnit.MILLISECONDS) + .transportProtocol(SiteToSiteTransportProtocol.valueOf(rpg.getTransportProtocol())) + .url(rpg.getTargetUris()) + .useCompression(remotePort.isUseCompression()) + .sslContext(sslContext) + .eventReporter(EventReporter.NO_OP) + .build(); + } + + @Override + public Set getRelationships() { + return Collections.emptySet(); + } + + @Override + protected FnConnectionContext getContext() { + return connectionContext; + } + + @Override + protected ComponentLog getLogger() { + return logger; + } + + @Override + public void shutdown() { + + } + + @Override + public void enqueueAll(final Queue list) { + inputQueue.addAll(list); + } + + @Override + public boolean runRecursive(final Queue queue) { + try { + final Transaction transaction = client.createTransaction(TransferDirection.SEND); + if (transaction == null) { + getLogger().error("Unable to create a transaction for Remote Process Group {} to send to port {}", new Object[] {url, name}); + return false; + } + + FnFlowFile flowFile; + while ((flowFile = inputQueue.poll()) != null) { + final DataPacket dataPacket = new StandardDataPacket(flowFile.getAttributes(), flowFile.getDataStream(), flowFile.getSize()); + transaction.send(dataPacket); + } + + transaction.confirm(); + transaction.complete(); + } catch (final Exception e) { + getLogger().error("Failed to send FlowFile via site-to-site", e); + return false; + } + + return true; + } + + @Override + public boolean isMaterializeContent() { + return false; + } +} diff --git a/nifi-fn/nifi-fn-core/src/main/java/org/apache/nifi/fn/core/FnRemoteOutputPort.java b/nifi-fn/nifi-fn-core/src/main/java/org/apache/nifi/fn/core/FnRemoteOutputPort.java new file mode 100644 index 0000000000..c649aaecfc --- /dev/null +++ b/nifi-fn/nifi-fn-core/src/main/java/org/apache/nifi/fn/core/FnRemoteOutputPort.java @@ -0,0 +1,154 @@ +/* + * 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.fn.core; + +import org.apache.nifi.events.EventReporter; +import org.apache.nifi.fn.bootstrap.InMemoryFlowFile; +import org.apache.nifi.logging.ComponentLog; +import org.apache.nifi.processor.DataUnit; +import org.apache.nifi.processor.Relationship; +import org.apache.nifi.registry.flow.BatchSize; +import org.apache.nifi.registry.flow.VersionedRemoteGroupPort; +import org.apache.nifi.registry.flow.VersionedRemoteProcessGroup; +import org.apache.nifi.remote.Transaction; +import org.apache.nifi.remote.TransferDirection; +import org.apache.nifi.remote.client.SiteToSiteClient; +import org.apache.nifi.remote.protocol.DataPacket; +import org.apache.nifi.remote.protocol.SiteToSiteTransportProtocol; +import org.apache.nifi.stream.io.StreamUtils; +import org.apache.nifi.util.FormatUtils; + +import javax.net.ssl.SSLContext; +import java.io.InputStream; +import java.util.Collections; +import java.util.LinkedList; +import java.util.Map; +import java.util.Queue; +import java.util.Set; +import java.util.concurrent.TimeUnit; + +public class FnRemoteOutputPort extends AbstractFnComponent { + private final SiteToSiteClient client; + private final String url; + private final String name; + + private final ComponentLog logger = new SLF4JComponentLog(this); + private final FnConnectionContext connectionContext = new FnPassThroughConnectionContext(); + + public FnRemoteOutputPort(final VersionedRemoteProcessGroup rpg, final VersionedRemoteGroupPort remotePort, final SSLContext sslContext) { + final String timeout = rpg.getCommunicationsTimeout(); + final long timeoutMillis = FormatUtils.getTimeDuration(timeout, TimeUnit.MILLISECONDS); + + url = rpg.getTargetUris(); + name = remotePort.getName(); + + final BatchSize batchSize = remotePort.getBatchSize(); + final int batchCount; + final long batchBytes; + final long batchMillis; + if (batchSize == null) { + batchCount = 1; + batchBytes = 1L; + batchMillis = 1L; + } else { + batchCount = batchSize.getCount() == null ? 1 : batchSize.getCount(); + batchBytes = batchSize.getSize() == null ? 1L : DataUnit.parseDataSize(batchSize.getSize(), DataUnit.B).longValue(); + batchMillis = batchSize.getDuration() == null ? 1L : FormatUtils.getTimeDuration(batchSize.getDuration(), TimeUnit.MILLISECONDS); + } + + client = new SiteToSiteClient.Builder() + .portName(remotePort.getName()) + .timeout(timeoutMillis, TimeUnit.MILLISECONDS) + .requestBatchCount(batchCount) + .requestBatchDuration(batchMillis, TimeUnit.MILLISECONDS) + .requestBatchSize(batchBytes) + .transportProtocol(SiteToSiteTransportProtocol.valueOf(rpg.getTransportProtocol())) + .url(rpg.getTargetUris()) + .sslContext(sslContext) + .useCompression(remotePort.isUseCompression()) + .eventReporter(EventReporter.NO_OP) + .build(); + } + + @Override + public Set getRelationships() { + return Collections.emptySet(); + } + + @Override + protected FnConnectionContext getContext() { + return connectionContext; + } + + @Override + protected ComponentLog getLogger() { + return logger; + } + + @Override + public void shutdown() { + + } + + @Override + public void enqueueAll(final Queue list) { + throw new UnsupportedOperationException("Cannot enqueue FlowFiles for a Remote Output Port"); + } + + @Override + public boolean runRecursive(final Queue queue) { + try { + final Transaction transaction = client.createTransaction(TransferDirection.RECEIVE); + if (transaction == null) { + getLogger().error("Unable to create a transaction for Remote Process Group {} to pull from port {}", new Object[]{url, name}); + return false; + } + + final Queue destinationQueue = new LinkedList<>(); + DataPacket dataPacket; + while ((dataPacket = transaction.receive()) != null) { + final Map attributes = dataPacket.getAttributes(); + final InputStream in = dataPacket.getData(); + final byte[] buffer = new byte[(int) dataPacket.getSize()]; + StreamUtils.fillBuffer(in, buffer); + + final FnFlowFile receivedFlowFile = new FnFlowFile(buffer, attributes, true); + destinationQueue.add(receivedFlowFile); + + for (final FnComponent childComponent : getChildren().get(Relationship.ANONYMOUS)) { + childComponent.enqueueAll(destinationQueue); + childComponent.runRecursive(queue); + } + + destinationQueue.clear(); + } + + transaction.confirm(); + transaction.complete(); + } catch (final Exception e) { + getLogger().error("Failed to receive FlowFile via site-to-site", e); + return false; + } + + return true; + } + + @Override + public boolean isMaterializeContent() { + return false; + } +} diff --git a/nifi-fn/src/main/java/org/apache/nifi/fn/core/FnStateManager.java b/nifi-fn/nifi-fn-core/src/main/java/org/apache/nifi/fn/core/FnStateManager.java similarity index 88% rename from nifi-fn/src/main/java/org/apache/nifi/fn/core/FnStateManager.java rename to nifi-fn/nifi-fn-core/src/main/java/org/apache/nifi/fn/core/FnStateManager.java index 37c16b8025..e3cdcf2e5b 100644 --- a/nifi-fn/src/main/java/org/apache/nifi/fn/core/FnStateManager.java +++ b/nifi-fn/nifi-fn-core/src/main/java/org/apache/nifi/fn/core/FnStateManager.java @@ -30,9 +30,10 @@ public class FnStateManager implements StateManager { private final AtomicInteger versionIndex = new AtomicInteger(0); private Map maps; //Local, Cluster - public FnStateManager(){ + + public FnStateManager() { this.maps = new HashMap<>(); - for(Scope s : Scope.values()){ + for (Scope s : Scope.values()) { this.maps.put(s, new FnStateMap(null, -1L)); } } @@ -44,9 +45,10 @@ public class FnStateManager implements StateManager { public synchronized StateMap getState(final Scope scope) { return maps.get(scope); } + public synchronized boolean replace(final StateMap oldValue, final Map newValue, final Scope scope) throws IOException { - if(oldValue == maps.get(scope)){ - maps.put(scope,new FnStateMap(newValue, versionIndex.incrementAndGet())); + if (oldValue == maps.get(scope)) { + maps.put(scope, new FnStateMap(newValue, versionIndex.incrementAndGet())); return true; } else { return false; @@ -54,6 +56,6 @@ public class FnStateManager implements StateManager { } public synchronized void clear(final Scope scope) { - setState(Collections. emptyMap(), scope); + setState(Collections.emptyMap(), scope); } } diff --git a/nifi-fn/src/main/java/org/apache/nifi/fn/core/FnStateMap.java b/nifi-fn/nifi-fn-core/src/main/java/org/apache/nifi/fn/core/FnStateMap.java similarity index 100% rename from nifi-fn/src/main/java/org/apache/nifi/fn/core/FnStateMap.java rename to nifi-fn/nifi-fn-core/src/main/java/org/apache/nifi/fn/core/FnStateMap.java diff --git a/nifi-fn/src/main/java/org/apache/nifi/fn/core/FnValidationContext.java b/nifi-fn/nifi-fn-core/src/main/java/org/apache/nifi/fn/core/FnValidationContext.java similarity index 95% rename from nifi-fn/src/main/java/org/apache/nifi/fn/core/FnValidationContext.java rename to nifi-fn/nifi-fn-core/src/main/java/org/apache/nifi/fn/core/FnValidationContext.java index b97375a475..b8faa7cab0 100644 --- a/nifi-fn/src/main/java/org/apache/nifi/fn/core/FnValidationContext.java +++ b/nifi-fn/nifi-fn-core/src/main/java/org/apache/nifi/fn/core/FnValidationContext.java @@ -27,7 +27,10 @@ import org.apache.nifi.controller.ControllerServiceLookup; import org.apache.nifi.expression.ExpressionLanguageCompiler; import org.apache.nifi.registry.VariableRegistry; -import java.util.*; +import java.util.HashMap; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; public class FnValidationContext implements ValidationContext { @@ -62,7 +65,7 @@ public class FnValidationContext implements ValidationContext { @Override public ValidationContext getControllerServiceValidationContext(final ControllerService controllerService) { - final FnProcessContext serviceProcessContext = new FnProcessContext(controllerService, lookup, null ,stateManager, variableRegistry); + final FnProcessContext serviceProcessContext = new FnProcessContext(controllerService, lookup, null, stateManager, variableRegistry); return new FnValidationContext(serviceProcessContext, lookup, stateManager, variableRegistry); } @@ -78,7 +81,7 @@ public class FnValidationContext implements ValidationContext { @Override public Map getAllProperties() { - final Map propValueMap = new LinkedHashMap<>(); + final Map propValueMap = new LinkedHashMap<>(); for (final Map.Entry entry : getProperties().entrySet()) { propValueMap.put(entry.getKey().getName(), entry.getValue()); } @@ -90,7 +93,6 @@ public class FnValidationContext implements ValidationContext { return processContext.getAnnotationData(); } - @Override public boolean isExpressionLanguagePresent(final String value) { if (value == null) { @@ -117,7 +119,6 @@ public class FnValidationContext implements ValidationContext { return this.lookup; } - @Override public boolean isValidationRequired(final ControllerService service) { return true; diff --git a/nifi-fn/src/main/java/org/apache/nifi/fn/core/ProvenanceCollector.java b/nifi-fn/nifi-fn-core/src/main/java/org/apache/nifi/fn/core/ProvenanceCollector.java similarity index 95% rename from nifi-fn/src/main/java/org/apache/nifi/fn/core/ProvenanceCollector.java rename to nifi-fn/nifi-fn-core/src/main/java/org/apache/nifi/fn/core/ProvenanceCollector.java index b497690a65..aa941767ff 100644 --- a/nifi-fn/src/main/java/org/apache/nifi/fn/core/ProvenanceCollector.java +++ b/nifi-fn/nifi-fn-core/src/main/java/org/apache/nifi/fn/core/ProvenanceCollector.java @@ -19,11 +19,17 @@ package org.apache.nifi.fn.core; import org.apache.nifi.flowfile.FlowFile; import org.apache.nifi.processor.Relationship; import org.apache.nifi.processor.exception.FlowFileHandlingException; -import org.apache.nifi.provenance.*; +import org.apache.nifi.provenance.ProvenanceEventBuilder; +import org.apache.nifi.provenance.ProvenanceEventRecord; +import org.apache.nifi.provenance.ProvenanceEventType; +import org.apache.nifi.provenance.ProvenanceReporter; +import org.apache.nifi.provenance.StandardProvenanceEventRecord; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.util.*; +import java.util.Collection; +import java.util.LinkedHashSet; +import java.util.Set; public class ProvenanceCollector implements ProvenanceReporter { private static final Logger logger = LoggerFactory.getLogger(ProvenanceCollector.class); @@ -48,8 +54,7 @@ public class ProvenanceCollector implements ProvenanceReporter { /** * Removes the given event from the reporter * - * @param event - * event + * @param event event */ void remove(final ProvenanceEventRecord event) { events.remove(event); @@ -77,10 +82,9 @@ public class ProvenanceCollector implements ProvenanceReporter { * ability to de-dupe events, since one or more events may be created by the * session itself, as well as by the Processor * - * @param parents - * parents - * @param child - * child + * @param parents parents + * @param child child + * * @return record */ ProvenanceEventRecord generateJoinEvent(final Collection parents, final FlowFile child) { @@ -124,11 +128,11 @@ public class ProvenanceCollector implements ProvenanceReporter { try { final ProvenanceEventRecord record = build(flowFile, ProvenanceEventType.RECEIVE) - .setTransitUri(transitUri) - .setSourceSystemFlowFileIdentifier(sourceSystemFlowFileIdentifier) - .setEventDuration(transmissionMillis) - .setDetails(details) - .build(); + .setTransitUri(transitUri) + .setSourceSystemFlowFileIdentifier(sourceSystemFlowFileIdentifier) + .setEventDuration(transmissionMillis) + .setDetails(details) + .build(); events.add(record); } catch (final Exception e) { logger.error("Failed to generate Provenance Event due to " + e); @@ -154,10 +158,10 @@ public class ProvenanceCollector implements ProvenanceReporter { try { final ProvenanceEventRecord record = build(flowFile, ProvenanceEventType.FETCH) - .setTransitUri(transitUri) - .setEventDuration(transmissionMillis) - .setDetails(details) - .build(); + .setTransitUri(transitUri) + .setEventDuration(transmissionMillis) + .setDetails(details) + .build(); events.add(record); } catch (final Exception e) { logger.error("Failed to generate Provenance Event due to " + e); @@ -229,7 +233,7 @@ public class ProvenanceCollector implements ProvenanceReporter { public void invokeRemoteProcess(FlowFile flowFile, String transitUri, String details) { try { final ProvenanceEventRecord record = build(flowFile, ProvenanceEventType.REMOTE_INVOCATION) - .setTransitUri(transitUri).setDetails(details).build(); + .setTransitUri(transitUri).setDetails(details).build(); events.add(record); } catch (final Exception e) { logger.error("Failed to generate Provenance Event due to " + e); diff --git a/nifi-fn/src/main/java/org/apache/nifi/fn/core/ReflectionUtils.java b/nifi-fn/nifi-fn-core/src/main/java/org/apache/nifi/fn/core/ReflectionUtils.java similarity index 53% rename from nifi-fn/src/main/java/org/apache/nifi/fn/core/ReflectionUtils.java rename to nifi-fn/nifi-fn-core/src/main/java/org/apache/nifi/fn/core/ReflectionUtils.java index cc3506e206..87e65d5a0d 100644 --- a/nifi-fn/src/main/java/org/apache/nifi/fn/core/ReflectionUtils.java +++ b/nifi-fn/nifi-fn-core/src/main/java/org/apache/nifi/fn/core/ReflectionUtils.java @@ -16,24 +16,12 @@ */ package org.apache.nifi.fn.core; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + import java.lang.annotation.Annotation; import java.lang.reflect.InvocationTargetException; import java.lang.reflect.Method; -import java.util.Set; -import java.util.UUID; - -import org.apache.nifi.bundle.Bundle; -import org.apache.nifi.controller.ControllerService; -import org.apache.nifi.controller.exception.ControllerServiceInstantiationException; -import org.apache.nifi.controller.exception.ProcessorInstantiationException; -import org.apache.nifi.nar.ExtensionManager; -import org.apache.nifi.nar.SystemBundle; -import org.apache.nifi.processor.Processor; -import org.apache.nifi.registry.flow.VersionedControllerService; -import org.apache.nifi.registry.flow.VersionedProcessor; -import org.apache.nifi.util.NiFiProperties; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; public class ReflectionUtils { @@ -50,12 +38,13 @@ public class ReflectionUtils { * @param annotation the annotation to look for * @param instance to invoke a method of * @param args to supply in a method call + * * @throws InvocationTargetException ite * @throws IllegalArgumentException iae * @throws IllegalAccessException if not allowed to invoke that method */ public static void invokeMethodsWithAnnotation(final Class annotation, final Object instance, final Object... args) - throws IllegalAccessException, IllegalArgumentException, InvocationTargetException { + throws IllegalAccessException, IllegalArgumentException, InvocationTargetException { for (final Method method : instance.getClass().getMethods()) { if (method.isAnnotationPresent(annotation)) { final boolean isAccessible = method.isAccessible(); @@ -65,15 +54,15 @@ public class ReflectionUtils { final Class[] argumentTypes = method.getParameterTypes(); if (argumentTypes.length > args.length) { throw new IllegalArgumentException(String.format("Unable to invoke method %1$s on %2$s because method expects %3$s parameters but only %4$s were given", - method.getName(), instance, argumentTypes.length, args.length)); + method.getName(), instance, argumentTypes.length, args.length)); } for (int i = 0; i < argumentTypes.length; i++) { final Class argType = argumentTypes[i]; if (!argType.isAssignableFrom(args[i].getClass())) { throw new IllegalArgumentException(String.format( - "Unable to invoke method %1$s on %2$s because method parameter %3$s is expected to be of type %4$s but argument passed was of type %5$s", - method.getName(), instance, i, argType, args[i].getClass())); + "Unable to invoke method %1$s on %2$s because method parameter %3$s is expected to be of type %4$s but argument passed was of type %5$s", + method.getName(), instance, i, argType, args[i].getClass())); } } @@ -81,9 +70,7 @@ public class ReflectionUtils { method.invoke(instance, args); } else { final Object[] argsToPass = new Object[argumentTypes.length]; - for (int i = 0; i < argsToPass.length; i++) { - argsToPass[i] = args[i]; - } + System.arraycopy(args, 0, argsToPass, 0, argsToPass.length); method.invoke(instance, argsToPass); } @@ -107,6 +94,7 @@ public class ReflectionUtils { * @param annotation the annotation to look for * @param instance to invoke a method of * @param args to supply in a method call + * * @return true if all appropriate methods were invoked and * returned without throwing an Exception, false if one of the * methods threw an Exception or could not be invoked; if false @@ -122,7 +110,7 @@ public class ReflectionUtils { final Class[] argumentTypes = method.getParameterTypes(); if (argumentTypes.length > args.length) { LOG.error("Unable to invoke method {} on {} because method expects {} parameters but only {} were given", - new Object[]{method.getName(), instance, argumentTypes.length, args.length}); + method.getName(), instance, argumentTypes.length, args.length); return false; } @@ -130,7 +118,7 @@ public class ReflectionUtils { final Class argType = argumentTypes[i]; if (!argType.isAssignableFrom(args[i].getClass())) { LOG.error("Unable to invoke method {} on {} because method parameter {} is expected to be of type {} but argument passed was of type {}", - new Object[]{method.getName(), instance, i, argType, args[i].getClass()}); + method.getName(), instance, i, argType, args[i].getClass()); return false; } } @@ -140,14 +128,12 @@ public class ReflectionUtils { method.invoke(instance, args); } else { final Object[] argsToPass = new Object[argumentTypes.length]; - for (int i = 0; i < argsToPass.length; i++) { - argsToPass[i] = args[i]; - } + System.arraycopy(args, 0, argsToPass, 0, argsToPass.length); method.invoke(instance, argsToPass); } } catch (final IllegalAccessException | IllegalArgumentException | InvocationTargetException t) { - LOG.error("Unable to invoke method {} on {} due to {}", new Object[]{method.getName(), instance, t}); + LOG.error("Unable to invoke method {} on {} due to {}", method.getName(), instance, t); LOG.error("", t); return false; } @@ -160,78 +146,5 @@ public class ReflectionUtils { } return true; } - public static ControllerService createControllerService(VersionedControllerService versionedControllerService) { - //org.apache.nifi.registry.flow.Bundle bundle = versionedControllerService.getBundle(); - //BundleCoordinate coordinate = new BundleCoordinate(bundle.getGroup(), bundle.getArtifact(), "1.7.1"); - //final Bundle processorBundle = ExtensionManager.getBundle(coordinate); - //if (processorBundle == null) { - // throw new ProcessorInstantiationException("Unable to find bundle for coordinate " + bundle.toString()); - //} - final Bundle systemBundle = SystemBundle.create(new NiFiProperties() { - @Override - public String getProperty(String s) { - if(s.equals("nifi.nar.library.directory")) - return "/usr/share/nifi-1.8.0/lib/"; - return null; - } - - @Override - public Set getPropertyKeys() { - return null; - } - }); - final ClassLoader ctxClassLoader = Thread.currentThread().getContextClassLoader(); - try { - //final ClassLoader detectedClassLoaderForInstance = ExtensionManager.createInstanceClassLoader(versionedControllerService.getType(), UUID.randomUUID().toString(), systemBundle, null); - final Class rawClass = Class.forName(versionedControllerService.getType(), true, ctxClassLoader); - //Thread.currentThread().setContextClassLoader(detectedClassLoaderForInstance); - - final Class processorClass = rawClass.asSubclass(ControllerService.class); - return processorClass.newInstance(); - } catch (final Throwable t) { - throw new ControllerServiceInstantiationException(versionedControllerService.getType(), t); - } finally { - if (ctxClassLoader != null) { - Thread.currentThread().setContextClassLoader(ctxClassLoader); - } - } - } - public static Processor createProcessor(VersionedProcessor versionedProcessor) throws ProcessorInstantiationException { - //org.apache.nifi.registry.flow.Bundle bundle = versionedProcessor.getBundle(); - //BundleCoordinate coordinate = new BundleCoordinate(bundle.getGroup(), bundle.getArtifact(), "1.8.0"); - //final Bundle processorBundle = ExtensionManager.getBundle(coordinate); - //if (processorBundle == null) { - // throw new ProcessorInstantiationException("Unable to find bundle for coordinate " + bundle.toString()); - //} - - final Bundle systemBundle = SystemBundle.create(new NiFiProperties() { - @Override - public String getProperty(String s) { - if(s.equals("nifi.nar.library.directory")) - return "/usr/share/nifi-1.8.0/lib/"; - return null; - } - - @Override - public Set getPropertyKeys() { - return null; - } - }); - final ClassLoader ctxClassLoader = Thread.currentThread().getContextClassLoader(); - try { - //final ClassLoader detectedClassLoaderForInstance = ExtensionManager.createInstanceClassLoader(versionedProcessor.getType(), UUID.randomUUID().toString(), systemBundle, null); - final Class rawClass = Class.forName(versionedProcessor.getType(), true, ctxClassLoader); - //Thread.currentThread().setContextClassLoader(detectedClassLoaderForInstance); - - final Class processorClass = rawClass.asSubclass(Processor.class); - return processorClass.newInstance(); - } catch (final Throwable t) { - throw new ProcessorInstantiationException(versionedProcessor.getType(), t); - } finally { - if (ctxClassLoader != null) { - Thread.currentThread().setContextClassLoader(ctxClassLoader); - } - } - } } diff --git a/nifi-fn/nifi-fn-core/src/main/java/org/apache/nifi/fn/core/RegistryUtil.java b/nifi-fn/nifi-fn-core/src/main/java/org/apache/nifi/fn/core/RegistryUtil.java new file mode 100644 index 0000000000..126886b876 --- /dev/null +++ b/nifi-fn/nifi-fn-core/src/main/java/org/apache/nifi/fn/core/RegistryUtil.java @@ -0,0 +1,160 @@ +/* + * 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.fn.core; + +import org.apache.nifi.authorization.user.NiFiUser; +import org.apache.nifi.registry.client.FlowClient; +import org.apache.nifi.registry.client.FlowSnapshotClient; +import org.apache.nifi.registry.client.NiFiRegistryClient; +import org.apache.nifi.registry.client.NiFiRegistryClientConfig; +import org.apache.nifi.registry.client.NiFiRegistryException; +import org.apache.nifi.registry.client.impl.JerseyNiFiRegistryClient; +import org.apache.nifi.registry.flow.VersionedFlowCoordinates; +import org.apache.nifi.registry.flow.VersionedFlowSnapshot; +import org.apache.nifi.registry.flow.VersionedProcessGroup; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.net.ssl.SSLContext; +import java.io.IOException; +import java.util.concurrent.TimeUnit; + +public class RegistryUtil { + private static final Logger logger = LoggerFactory.getLogger(RegistryUtil.class); + + private final String registryUrl; + private NiFiRegistryClient registryClient; + private final SSLContext sslContext; + + public RegistryUtil(final String registryUrl, final SSLContext sslContext) { + this.registryUrl = registryUrl; + this.sslContext = sslContext; + } + + + public VersionedFlowSnapshot getFlowByID(String bucketID, String flowID) throws IOException, NiFiRegistryException { + return getFlowByID(bucketID, flowID, -1); + } + + public VersionedFlowSnapshot getFlowByID(String bucketID, String flowID, int versionID) throws IOException, NiFiRegistryException { + if (versionID == -1) { + // TODO: Have to support providing some sort of user + versionID = getLatestVersion(bucketID, flowID, null); + } + + logger.debug("Fetching flow Bucket={}, Flow={}, Version={}, FetchRemoteFlows=true", bucketID, flowID, versionID); + final long start = System.nanoTime(); + final VersionedFlowSnapshot snapshot = getFlowContents(bucketID, flowID, versionID, true, null); + final long millis = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - start); + logger.info("Successfully fetched flow from registry in {} millis", millis); + + return snapshot; + } + + private int getLatestVersion(final String bucketId, final String flowId, final NiFiUser user) throws IOException, NiFiRegistryException { + return (int) getFlowClient(user).get(bucketId, flowId).getVersionCount(); + } + + private FlowClient getFlowClient(final NiFiUser user) { + final String identity = getIdentity(user); + final NiFiRegistryClient registryClient = getRegistryClient(); + final FlowClient flowClient = identity == null ? registryClient.getFlowClient() : registryClient.getFlowClient(identity); + return flowClient; + } + + private FlowSnapshotClient getFlowSnapshotClient(final NiFiUser user) { + final String identity = getIdentity(user); + final NiFiRegistryClient registryClient = getRegistryClient(); + final FlowSnapshotClient snapshotClient = identity == null ? registryClient.getFlowSnapshotClient() : registryClient.getFlowSnapshotClient(identity); + return snapshotClient; + } + + private synchronized NiFiRegistryClient getRegistryClient() { + if (registryClient != null) { + return registryClient; + } + + final NiFiRegistryClientConfig config = new NiFiRegistryClientConfig.Builder() + .connectTimeout(30000) + .readTimeout(30000) + .sslContext(sslContext) + .baseUrl(registryUrl) + .build(); + + registryClient = new JerseyNiFiRegistryClient.Builder() + .config(config) + .build(); + + return registryClient; + } + + private String getIdentity(final NiFiUser user) { + return (user == null || user.isAnonymous()) ? null : user.getIdentity(); + } + + public VersionedFlowSnapshot getFlowContents(final String bucketId, final String flowId, final int version, final boolean fetchRemoteFlows, final NiFiUser user) + throws IOException, NiFiRegistryException { + + final FlowSnapshotClient snapshotClient = getFlowSnapshotClient(user); + final VersionedFlowSnapshot flowSnapshot = snapshotClient.get(bucketId, flowId, version); + + if (fetchRemoteFlows) { + final VersionedProcessGroup contents = flowSnapshot.getFlowContents(); + for (final VersionedProcessGroup child : contents.getProcessGroups()) { + populateVersionedContentsRecursively(child, user); + } + } + + return flowSnapshot; + } + + + private void populateVersionedContentsRecursively(final VersionedProcessGroup group, final NiFiUser user) throws NiFiRegistryException, IOException { + if (group == null) { + return; + } + + final VersionedFlowCoordinates coordinates = group.getVersionedFlowCoordinates(); + if (coordinates != null) { + final String registryUrl = coordinates.getRegistryUrl(); + final String bucketId = coordinates.getBucketId(); + final String flowId = coordinates.getFlowId(); + final int version = coordinates.getVersion(); + + final RegistryUtil subFlowUtil = new RegistryUtil(registryUrl, sslContext); + final VersionedFlowSnapshot snapshot = subFlowUtil.getFlowByID(bucketId, flowId, version); + final VersionedProcessGroup contents = snapshot.getFlowContents(); + + group.setComments(contents.getComments()); + group.setConnections(contents.getConnections()); + group.setControllerServices(contents.getControllerServices()); + group.setFunnels(contents.getFunnels()); + group.setInputPorts(contents.getInputPorts()); + group.setLabels(contents.getLabels()); + group.setOutputPorts(contents.getOutputPorts()); + group.setProcessGroups(contents.getProcessGroups()); + group.setProcessors(contents.getProcessors()); + group.setRemoteProcessGroups(contents.getRemoteProcessGroups()); + group.setVariables(contents.getVariables()); + coordinates.setLatest(snapshot.isLatest()); + } + + for (final VersionedProcessGroup child : group.getProcessGroups()) { + populateVersionedContentsRecursively(child, user); + } + } +} diff --git a/nifi-fn/src/main/java/org/apache/nifi/fn/core/SLF4JComponentLog.java b/nifi-fn/nifi-fn-core/src/main/java/org/apache/nifi/fn/core/SLF4JComponentLog.java similarity index 100% rename from nifi-fn/src/main/java/org/apache/nifi/fn/core/SLF4JComponentLog.java rename to nifi-fn/nifi-fn-core/src/main/java/org/apache/nifi/fn/core/SLF4JComponentLog.java diff --git a/nifi-fn/nifi-fn-core/src/main/java/org/apache/nifi/fn/runtimes/Program.java b/nifi-fn/nifi-fn-core/src/main/java/org/apache/nifi/fn/runtimes/Program.java new file mode 100644 index 0000000000..f0ff71739a --- /dev/null +++ b/nifi-fn/nifi-fn-core/src/main/java/org/apache/nifi/fn/runtimes/Program.java @@ -0,0 +1,164 @@ +/* + * 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.fn.runtimes; + +import org.apache.nifi.fn.bootstrap.InMemoryFlowFile; +import org.apache.nifi.fn.bootstrap.RunnableFlow; +import org.apache.nifi.fn.bootstrap.RunnableFlowFactory; +import org.apache.nifi.fn.core.FnFlow; +import org.apache.nifi.fn.runtimes.openwhisk.NiFiFnOpenWhiskAction; +import org.apache.nifi.fn.runtimes.yarn.YARNServiceUtil; + +import java.io.File; +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Paths; +import java.util.Arrays; +import java.util.LinkedList; +import java.util.List; +import java.util.Queue; + +public class Program { + + public static final String RUN_FROM_REGISTRY = "RunFromRegistry"; + public static final String RUN_YARN_SERVICE_FROM_REGISTRY = "RunYARNServiceFromRegistry"; + public static final String RUN_OPENWHISK_ACTION_SERVER = "RunOpenwhiskActionServer"; + + + public static void launch(final String[] args, final ClassLoader systemClassLoader, final File narWorkingDirectory) throws Exception { + if (args.length == 0) { + printUsage(); + System.exit(1); + } else if (args[0].equals(RUN_FROM_REGISTRY) && (args[1].equalsIgnoreCase("Once") || args[1].equalsIgnoreCase("Continuous")) && args.length >= 4) { + runLocal(args, systemClassLoader, narWorkingDirectory); + } else if (args[0].equals(RUN_YARN_SERVICE_FROM_REGISTRY) && args.length >= 7) { + runOnYarn(args); + } else if (args[0].equals(RUN_OPENWHISK_ACTION_SERVER) && args.length == 2) { + runOnOpenWhisk(args); + } else { + System.out.println("Invalid input: " + String.join(",", args)); + printUsage(); + System.exit(1); + } + } + + private static void runOnOpenWhisk(final String[] args) throws IOException { + NiFiFnOpenWhiskAction action = new NiFiFnOpenWhiskAction(Integer.parseInt(args[1])); + action.start(); + } + + private static void runOnYarn(final String[] args) throws IOException { + String YARNUrl = args[1]; + String imageName = args[2]; + String serviceName = args[3]; + int numberOfContainers = Integer.parseInt(args[4]); + List launchCommand = Arrays.asList(RUN_FROM_REGISTRY, "Continuous"); + + if (args[5].equals("--file")) { + launchCommand.add("--json"); + launchCommand.add(new String(Files.readAllBytes(Paths.get(args[6])))); + } else if (args[5].equals("--json")) { + launchCommand.add("--json"); + launchCommand.add(args[6]); + } + + if (args.length >= 9) { + for (int i = 5; i < args.length; i++) { + launchCommand.add(args[i]); + } + } else { + System.out.println("Invalid input: " + String.join(",", args)); + printUsage(); + System.exit(1); + } + + StringBuilder message = new StringBuilder(); + YARNServiceUtil yarnServiceUtil = new YARNServiceUtil(YARNUrl, imageName); + yarnServiceUtil.launchYARNService(serviceName, numberOfContainers, launchCommand.toArray(new String[0]), message); + System.out.println(message); + } + + private static void runLocal(final String[] args, final ClassLoader systemClassLoader, final File narWorkingDirectory) throws Exception { + final boolean once = args[1].equalsIgnoreCase("Once"); + + final RunnableFlow flow; + if (args[2].equals("--file")) { + flow = RunnableFlowFactory.fromJsonFile(args[3], systemClassLoader, narWorkingDirectory); + } else if (args[2].equals("--json")) { + flow = RunnableFlowFactory.fromJson(args[3]); + } else if (args.length >= 5) { + flow = RunnableFlowFactory.fromCommandLineArgs(args); + } else { + System.out.println("Invalid input: " + String.join(",", args)); + printUsage(); + System.exit(1); + return; + } + + // Run Flow + final Queue outputFlowFiles = new LinkedList<>(); + final boolean successful; + if (once) { + successful = flow.runOnce(outputFlowFiles); + } else { + successful = flow.run(outputFlowFiles); //Run forever + } + + if (successful) { + System.out.println("Flow Succeeded"); + outputFlowFiles.forEach(f -> System.out.println(f.toStringFull())); + } else { + System.out.println("Flow Failed"); + outputFlowFiles.forEach(f -> System.out.println(f.toStringFull())); + System.exit(1); + } + } + + + private static void printUsage() { + System.out.println("Usage:"); + System.out.println(" 1) " + RUN_FROM_REGISTRY + " [Once|Continuous] [] []"); + System.out.println(" " + RUN_FROM_REGISTRY + " [Once|Continuous] --json "); + System.out.println(" " + RUN_FROM_REGISTRY + " [Once|Continuous] --file "); + System.out.println(); + System.out.println(" 2) " + RUN_YARN_SERVICE_FROM_REGISTRY + " <# of Containers> \\"); + System.out.println(" [] []"); + System.out.println(" " + RUN_YARN_SERVICE_FROM_REGISTRY + " <# of Containers> --json "); + System.out.println(" " + RUN_YARN_SERVICE_FROM_REGISTRY + " <# of Containers> --file "); + System.out.println(); + System.out.println(" 3) " + RUN_OPENWHISK_ACTION_SERVER + " "); + System.out.println(); + System.out.println("Examples:"); + System.out.println(" 1) " + RUN_FROM_REGISTRY + " Once http://172.0.0.1:61080 e53b8a0d-5c85-4fcd-912a-1c549a586c83 6cf8277a-c402-4957-8623-0fa9890dd45d \\"); + System.out.println(" \"DestinationDirectory-/tmp/nififn/output2/\" \"\" \"absolute.path-/tmp/nififn/input/;filename-test.txt\" \"absolute.path-/tmp/nififn/input/;" + + "filename-test2.txt\""); + System.out.println(" 2) " + RUN_FROM_REGISTRY + " Once http://172.0.0.1:61080 e53b8a0d-5c85-4fcd-912a-1c549a586c83 6cf8277a-c402-4957-8623-0fa9890dd45d \\"); + System.out.println(" \"DestinationDirectory-/tmp/nififn/output2/\" \"f25c9204-6c95-3aa9-b0a8-c556f5f61849\" \"absolute.path-/tmp/nififn/input/;filename-test.txt\""); + System.out.println(" 3) " + RUN_YARN_SERVICE_FROM_REGISTRY + " http://127.0.0.1:8088 nifi-fn:latest kafka-to-solr 3 --file kafka-to-solr.json"); + System.out.println(" 4) " + RUN_OPENWHISK_ACTION_SERVER + " 8080"); + System.out.println(); + System.out.println("Notes:"); + System.out.println(" 1) will be split on ';' and '-' then injected into the flow using the variable registry interface."); + System.out.println(" 2) will be split on ';'. FlowFiles routed to matching output ports will immediately fail the flow."); + System.out.println(" 3) will be split on ';' and '-' then injected into the flow using the \"" + FnFlow.CONTENT + "\" field as the FlowFile content."); + System.out.println(" 4) Multiple arguments can be provided."); + System.out.println(" 5) The configuration file must be in JSON format. "); + System.out.println(" 6) When providing configurations via JSON, the following attributes must be provided: " + FnFlow.REGISTRY + ", " + FnFlow.BUCKETID + ", " + FnFlow.FLOWID + "."); + System.out.println(" All other attributes will be passed to the flow using the variable registry interface"); + System.out.println(); + } +} diff --git a/nifi-fn/src/main/java/org/apache/nifi/fn/runtimes/OpenWhisk/JavaAction.java b/nifi-fn/nifi-fn-core/src/main/java/org/apache/nifi/fn/runtimes/openwhisk/JavaAction.java similarity index 80% rename from nifi-fn/src/main/java/org/apache/nifi/fn/runtimes/OpenWhisk/JavaAction.java rename to nifi-fn/nifi-fn-core/src/main/java/org/apache/nifi/fn/runtimes/openwhisk/JavaAction.java index 1b8694a335..c14dbbefc4 100644 --- a/nifi-fn/src/main/java/org/apache/nifi/fn/runtimes/OpenWhisk/JavaAction.java +++ b/nifi-fn/nifi-fn-core/src/main/java/org/apache/nifi/fn/runtimes/openwhisk/JavaAction.java @@ -14,11 +14,11 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.nifi.fn.runtimes.OpenWhisk; +package org.apache.nifi.fn.runtimes.openwhisk; import com.google.gson.JsonObject; +import org.apache.nifi.fn.bootstrap.InMemoryFlowFile; import org.apache.nifi.fn.core.FnFlow; -import org.apache.nifi.fn.core.FnFlowFile; import java.io.PrintWriter; import java.io.StringWriter; @@ -34,20 +34,21 @@ public class JavaAction { JsonObject result = new JsonObject(); try { FnFlow flow = FnFlow.createAndEnqueueFromJSON(args); - Queue output = new LinkedList<>(); + Queue output = new LinkedList<>(); boolean successful = flow.runOnce(output); StringBuilder response = new StringBuilder(); - for(FnFlowFile file : output) - response.append("\n").append(file); + for (InMemoryFlowFile flowFile : output) { + response.append("\n").append(flowFile); + } result.addProperty("success", successful); result.addProperty("message", response.toString()); - } catch (Exception ex){ + } catch (Exception ex) { StringWriter sw = new StringWriter(); ex.printStackTrace(new PrintWriter(sw)); - result.addProperty("success",false); - result.addProperty("message", "Flow exception: "+ex.getMessage()+"--"+sw.toString()); + result.addProperty("success", false); + result.addProperty("message", "Flow exception: " + ex.getMessage() + "--" + sw.toString()); } return result; } diff --git a/nifi-fn/src/main/java/org/apache/nifi/fn/runtimes/OpenWhisk/NiFiFnOpenWhiskAction.java b/nifi-fn/nifi-fn-core/src/main/java/org/apache/nifi/fn/runtimes/openwhisk/NiFiFnOpenWhiskAction.java similarity index 84% rename from nifi-fn/src/main/java/org/apache/nifi/fn/runtimes/OpenWhisk/NiFiFnOpenWhiskAction.java rename to nifi-fn/nifi-fn-core/src/main/java/org/apache/nifi/fn/runtimes/openwhisk/NiFiFnOpenWhiskAction.java index ad91333904..da71da76e7 100644 --- a/nifi-fn/src/main/java/org/apache/nifi/fn/runtimes/OpenWhisk/NiFiFnOpenWhiskAction.java +++ b/nifi-fn/nifi-fn-core/src/main/java/org/apache/nifi/fn/runtimes/openwhisk/NiFiFnOpenWhiskAction.java @@ -14,17 +14,21 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.nifi.fn.runtimes.OpenWhisk; +package org.apache.nifi.fn.runtimes.openwhisk; import com.google.gson.JsonObject; import com.google.gson.JsonParser; import com.sun.net.httpserver.HttpExchange; import com.sun.net.httpserver.HttpHandler; import com.sun.net.httpserver.HttpServer; +import org.apache.nifi.fn.bootstrap.InMemoryFlowFile; import org.apache.nifi.fn.core.FnFlow; -import org.apache.nifi.fn.core.FnFlowFile; -import java.io.*; +import java.io.BufferedReader; +import java.io.IOException; +import java.io.InputStream; +import java.io.InputStreamReader; +import java.io.OutputStream; import java.net.InetSocketAddress; import java.nio.charset.StandardCharsets; import java.util.LinkedList; @@ -72,23 +76,23 @@ public class NiFiFnOpenWhiskAction { private class InitHandler implements HttpHandler { public void handle(HttpExchange t) throws IOException { initialized = true; - writeResponse(t, 200,"Initialized"); + writeResponse(t, 200, "Initialized"); - InputStream is = t.getRequestBody(); + InputStream is = t.getRequestBody(); JsonParser parser = new JsonParser(); JsonObject body = parser.parse(new BufferedReader(new InputStreamReader(is, StandardCharsets.UTF_8))).getAsJsonObject(); System.out.println("Init input: " + body); String code = body.get("value").getAsJsonObject().get("code").getAsString(); - if(code.equals("GENERIC")){ + if (code.equals("GENERIC")) { initialized = true; - writeResponse(t, 200,"Initialized Generic Action"); + writeResponse(t, 200, "Initialized Generic Action"); } else { JsonObject flowDefinition = parser.parse(code).getAsJsonObject(); try { flow = FnFlow.createAndEnqueueFromJSON(flowDefinition); initialized = true; - writeResponse(t, 200, "Initialized "+flow); + writeResponse(t, 200, "Initialized " + flow); } catch (Exception e) { e.printStackTrace(System.err); writeResponse(t, 400, "Error: " + e.getMessage()); @@ -116,7 +120,9 @@ public class NiFiFnOpenWhiskAction { "action_name":"/guest/nififn", "deadline":"1541729057462", "api_key":"23bc46b1-71f6-4ed5-8c54-816aa4f8c502:123zO3xZCLrMN6v2BKK1dXYFpXlPkccOFqm12CdAsMgRU4VrNZ9lyGVCGuMDGIwP", - "value":{"registry":"http://172.26.224.116:61080","SourceCluster":"hdfs://172.26.224.119:8020","SourceFile":"test.txt","SourceDirectory":"hdfs://172.26.224.119:8020/tmp/nififn/input/","flow":"6cf8277a-c402-4957-8623-0fa9890dd45d","bucket":"e53b8a0d-5c85-4fcd-912a-1c549a586c83","DestinationDirectory":"hdfs://172.26.224.119:8020/tmp/nififn/output"}, + "value":{"registry":"http://172.26.224.116:61080","SourceCluster":"hdfs://172.26.224.119:8020","SourceFile":"test.txt", + "SourceDirectory":"hdfs://172.26.224.119:8020/tmp/nififn/input/","flow":"6cf8277a-c402-4957-8623-0fa9890dd45d","bucket":"e53b8a0d-5c85-4fcd-912a-1c549a586c83", + "DestinationDirectory":"hdfs://172.26.224.119:8020/tmp/nififn/output"}, "namespace":"guest" } input headers: @@ -129,11 +135,10 @@ public class NiFiFnOpenWhiskAction { Content-length: [595] */ - // Run Flow - Queue output = new LinkedList<>(); + Queue output = new LinkedList<>(); boolean successful; - if(flow == null) { + if (flow == null) { FnFlow tempFlow = FnFlow.createAndEnqueueFromJSON(inputObject); successful = tempFlow.runOnce(output); } else { @@ -142,8 +147,10 @@ public class NiFiFnOpenWhiskAction { } StringBuilder response = new StringBuilder(); - for(FnFlowFile file : output) - response.append("\n").append(file); + for (InMemoryFlowFile flowFile : output) { + response.append("\n").append(flowFile); + } + NiFiFnOpenWhiskAction.writeResponse(t, successful ? 200 : 400, response.toString()); } catch (Exception e) { diff --git a/nifi-fn/src/main/java/org/apache/nifi/fn/runtimes/YARN/YARNServiceUtil.java b/nifi-fn/nifi-fn-core/src/main/java/org/apache/nifi/fn/runtimes/yarn/YARNServiceUtil.java similarity index 73% rename from nifi-fn/src/main/java/org/apache/nifi/fn/runtimes/YARN/YARNServiceUtil.java rename to nifi-fn/nifi-fn-core/src/main/java/org/apache/nifi/fn/runtimes/yarn/YARNServiceUtil.java index f4d56d44f2..b6ebf41253 100644 --- a/nifi-fn/src/main/java/org/apache/nifi/fn/runtimes/YARN/YARNServiceUtil.java +++ b/nifi-fn/nifi-fn-core/src/main/java/org/apache/nifi/fn/runtimes/yarn/YARNServiceUtil.java @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.nifi.fn.runtimes.YARN; +package org.apache.nifi.fn.runtimes.yarn; import com.google.gson.JsonArray; import com.google.gson.JsonObject; @@ -30,47 +30,47 @@ import java.nio.charset.StandardCharsets; public class YARNServiceUtil { private final String YARNUrl; private final String imageName; - public YARNServiceUtil(String YARNUrl, String imageName){ + + public YARNServiceUtil(String YARNUrl, String imageName) { this.YARNUrl = YARNUrl; this.imageName = imageName; } - public boolean launchYARNService(String name, int containerCount, String[] launchCommand, StringBuilder outMessage){ + + public boolean launchYARNService(String name, int containerCount, String[] launchCommand, StringBuilder outMessage) { JsonObject spec = new JsonObject(); - spec.addProperty("name", name.substring(0,25)); + spec.addProperty("name", name.substring(0, 25)); spec.addProperty("version", "1.0.0"); - spec.addProperty("description", "NiFi-Fn service launched with the following command: "+String.join(",",launchCommand)); + spec.addProperty("description", "NiFi-Fn service launched with the following command: " + String.join(",", launchCommand)); JsonObject component = new JsonObject(); - component.addProperty("name","mc"); - component.addProperty("number_of_containers",containerCount); + component.addProperty("name", "mc"); + component.addProperty("number_of_containers", containerCount); JsonObject artifact = new JsonObject(); - artifact.addProperty("id",this.imageName); - artifact.addProperty("type","DOCKER"); - component.add("artifact",artifact); + artifact.addProperty("id", this.imageName); + artifact.addProperty("type", "DOCKER"); + component.add("artifact", artifact); - component.addProperty("launch_command",String.join(",",launchCommand)); + component.addProperty("launch_command", String.join(",", launchCommand)); JsonObject resource = new JsonObject(); - resource.addProperty("cpus",1); - resource.addProperty("memory","256"); - component.add("resource",resource); - + resource.addProperty("cpus", 1); + resource.addProperty("memory", "256"); + component.add("resource", resource); JsonObject env = new JsonObject(); - env.addProperty("YARN_CONTAINER_RUNTIME_DOCKER_RUN_OVERRIDE_DISABLE","true"); + env.addProperty("YARN_CONTAINER_RUNTIME_DOCKER_RUN_OVERRIDE_DISABLE", "true"); JsonObject configuration = new JsonObject(); - configuration.add("env",env); - component.add("configuration",configuration); + configuration.add("env", env); + component.add("configuration", configuration); JsonArray components = new JsonArray(); components.add(component); spec.add("components", components); - HttpPost request = new HttpPost( - this.YARNUrl+"/app/v1/services?user.name="+System.getProperty("user.name") + this.YARNUrl + "/app/v1/services?user.name=" + System.getProperty("user.name") ); try { diff --git a/nifi-fn/src/test/java/org/apache/nifi/fn/core/BatchTest.java b/nifi-fn/nifi-fn-core/src/test/java/org/apache/nifi/fn/core/BatchTest.java similarity index 74% rename from nifi-fn/src/test/java/org/apache/nifi/fn/core/BatchTest.java rename to nifi-fn/nifi-fn-core/src/test/java/org/apache/nifi/fn/core/BatchTest.java index a715731b30..cdf9f43d65 100644 --- a/nifi-fn/src/test/java/org/apache/nifi/fn/core/BatchTest.java +++ b/nifi-fn/nifi-fn-core/src/test/java/org/apache/nifi/fn/core/BatchTest.java @@ -16,6 +16,8 @@ */ package org.apache.nifi.fn.core; +import org.apache.nifi.fn.bootstrap.InMemoryFlowFile; +import org.apache.nifi.processor.Processor; import org.apache.nifi.processors.standard.GetFile; import org.apache.nifi.processors.standard.PutFile; import org.apache.nifi.processors.standard.ReplaceText; @@ -25,12 +27,14 @@ import org.apache.nifi.registry.VariableRegistry; import java.io.File; import java.io.FileOutputStream; import java.io.PrintStream; +import java.lang.reflect.InvocationTargetException; import java.nio.charset.StandardCharsets; import java.nio.file.Files; import java.nio.file.Paths; import java.util.LinkedList; import java.util.List; import java.util.Queue; +import java.util.UUID; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; @@ -56,21 +60,24 @@ public class BatchTest { /////////////////////////////////////////// // Build Flow /////////////////////////////////////////// - FnProcessorWrapper getFile = new FnProcessorWrapper(new GetFile(), null, serviceLookup, registry, materializeData); + FnProcessorWrapper getFile = new FnProcessorWrapper(UUID.randomUUID().toString(), new GetFile(), null, serviceLookup, registry, materializeData, ClassLoader.getSystemClassLoader()); getFile.setProperty(GetFile.DIRECTORY,"/tmp/nififn/input/"); getFile.setProperty(GetFile.FILE_FILTER,"test.txt"); getFile.setProperty(GetFile.KEEP_SOURCE_FILE,"true"); - FnProcessorWrapper splitText = getFile.addChild(new SplitText(), GetFile.REL_SUCCESS); + FnProcessorWrapper splitText = wrapProcessor(new SplitText(), serviceLookup, registry); + getFile.addChild(splitText, GetFile.REL_SUCCESS); splitText.setProperty(SplitText.LINE_SPLIT_COUNT,"1"); splitText.addAutoTermination(SplitText.REL_FAILURE); splitText.addAutoTermination(SplitText.REL_ORIGINAL); - FnProcessorWrapper replaceText = splitText.addChild(new ReplaceText(), SplitText.REL_SPLITS); + FnProcessorWrapper replaceText = wrapProcessor(new ReplaceText(), serviceLookup, registry); + splitText.addChild(replaceText, SplitText.REL_SPLITS); replaceText.setProperty(ReplaceText.REPLACEMENT_VALUE,"$1!!!"); replaceText.addAutoTermination(ReplaceText.REL_FAILURE); - FnProcessorWrapper putFile = replaceText.addChild(new PutFile(), ReplaceText.REL_SUCCESS); + FnProcessorWrapper putFile = wrapProcessor(new PutFile(), serviceLookup, registry); + replaceText.addChild(putFile, ReplaceText.REL_SUCCESS); putFile.addAutoTermination(PutFile.REL_FAILURE); putFile.addAutoTermination(PutFile.REL_SUCCESS); putFile.setProperty(PutFile.DIRECTORY,"/tmp/nififn/output"); @@ -81,7 +88,7 @@ public class BatchTest { /////////////////////////////////////////// FnFlow flow = new FnFlow(getFile); - Queue output = new LinkedList<>(); + Queue output = new LinkedList<>(); boolean successful = flow.runOnce(output); /////////////////////////////////////////// @@ -97,4 +104,9 @@ public class BatchTest { assertEquals(1,lines.size()); assertEquals("hello world!!!", lines.get(0)); } + + private FnProcessorWrapper wrapProcessor(final Processor processor, FnControllerServiceLookup serviceLookup, final VariableRegistry registry) + throws InvocationTargetException, IllegalAccessException { + return new FnProcessorWrapper(UUID.randomUUID().toString(), processor, null, serviceLookup, registry, true, ClassLoader.getSystemClassLoader()); + } } \ No newline at end of file diff --git a/nifi-fn/src/test/java/org/apache/nifi/fn/core/RegistryTest.java b/nifi-fn/nifi-fn-core/src/test/java/org/apache/nifi/fn/core/RegistryTest.java similarity index 100% rename from nifi-fn/src/test/java/org/apache/nifi/fn/core/RegistryTest.java rename to nifi-fn/nifi-fn-core/src/test/java/org/apache/nifi/fn/core/RegistryTest.java diff --git a/nifi-fn/src/test/java/org/apache/nifi/fn/core/StreamingTest.java b/nifi-fn/nifi-fn-core/src/test/java/org/apache/nifi/fn/core/StreamingIT.java similarity index 78% rename from nifi-fn/src/test/java/org/apache/nifi/fn/core/StreamingTest.java rename to nifi-fn/nifi-fn-core/src/test/java/org/apache/nifi/fn/core/StreamingIT.java index cb40dd5af0..3ec25116cd 100644 --- a/nifi-fn/src/test/java/org/apache/nifi/fn/core/StreamingTest.java +++ b/nifi-fn/nifi-fn-core/src/test/java/org/apache/nifi/fn/core/StreamingIT.java @@ -16,6 +16,8 @@ */ package org.apache.nifi.fn.core; +import org.apache.nifi.fn.bootstrap.InMemoryFlowFile; +import org.apache.nifi.processor.Processor; import org.apache.nifi.processor.Relationship; import org.apache.nifi.processors.standard.PutFile; import org.apache.nifi.processors.standard.ReplaceText; @@ -35,12 +37,13 @@ import java.util.LinkedList; import java.util.List; import java.util.Queue; import java.util.Set; +import java.util.UUID; import java.util.concurrent.atomic.AtomicBoolean; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; -public class StreamingTest { +public class StreamingIT { @org.junit.Test public void Scenario1_Test() throws InvocationTargetException, IllegalAccessException, IOException, InterruptedException { @@ -60,36 +63,38 @@ public class StreamingTest { /////////////////////////////////////////// // Build Flow /////////////////////////////////////////// - FnProcessorWrapper tailFile = new FnProcessorWrapper(new TailFile(), null, serviceLookup, registry, materializeData); + FnProcessorWrapper tailFile = wrapProcessor(new TailFile(), serviceLookup, registry); tailFile.setProperty("File to Tail","/tmp/nififn/input/test.txt"); - Set relationships = tailFile.getProcessor().getRelationships(); + Set relationships = tailFile.getRelationships(); Relationship tailFile_Success = relationships.stream().filter(r->r.getName().equals("success")).findFirst().get(); - FnProcessorWrapper splitText = tailFile.addChild(new SplitText(), tailFile_Success); + FnProcessorWrapper splitText = wrapProcessor(new SplitText(), serviceLookup, registry); + tailFile.addChild(splitText, tailFile_Success); splitText.setProperty(SplitText.LINE_SPLIT_COUNT,"1"); splitText.addAutoTermination(SplitText.REL_FAILURE); splitText.addAutoTermination(SplitText.REL_ORIGINAL); - FnProcessorWrapper replaceText = splitText.addChild(new ReplaceText(), SplitText.REL_SPLITS); + FnProcessorWrapper replaceText = wrapProcessor(new ReplaceText(), serviceLookup, registry); + splitText.addChild(replaceText, SplitText.REL_SPLITS); replaceText.setProperty(ReplaceText.REPLACEMENT_VALUE,"$1!!!"); replaceText.addAutoTermination(ReplaceText.REL_FAILURE); - FnProcessorWrapper putFile = replaceText.addChild(new PutFile(), ReplaceText.REL_SUCCESS); + FnProcessorWrapper putFile = wrapProcessor(new PutFile(), serviceLookup, registry); + replaceText.addChild(putFile, ReplaceText.REL_SUCCESS); putFile.addAutoTermination(PutFile.REL_FAILURE); putFile.addAutoTermination(PutFile.REL_SUCCESS); putFile.setProperty(PutFile.DIRECTORY,"/tmp/nififn/output/"); putFile.setProperty(PutFile.CONFLICT_RESOLUTION, PutFile.REPLACE_RESOLUTION); - /////////////////////////////////////////// // Run Flow /////////////////////////////////////////// FnFlow flow = new FnFlow(tailFile); - Queue output = new LinkedList<>(); + Queue output = new LinkedList<>(); AtomicBoolean successful = new AtomicBoolean(true); Thread t = new Thread(()-> successful.set(flow.run(output)) @@ -122,4 +127,10 @@ public class StreamingTest { assertEquals(1,lines.size()); assertEquals("hello world!!!", lines.get(0)); } + + private FnProcessorWrapper wrapProcessor(final Processor processor, FnControllerServiceLookup serviceLookup, final VariableRegistry registry) + throws InvocationTargetException, IllegalAccessException { + return new FnProcessorWrapper(UUID.randomUUID().toString(), processor, null, serviceLookup, registry, true, ClassLoader.getSystemClassLoader()); + } + } diff --git a/nifi-fn/nifi-fn-nar/pom.xml b/nifi-fn/nifi-fn-nar/pom.xml new file mode 100644 index 0000000000..1b216dc31b --- /dev/null +++ b/nifi-fn/nifi-fn-nar/pom.xml @@ -0,0 +1,38 @@ + + + + + + nifi-fn + org.apache.nifi + 1.10.0-SNAPSHOT + + 4.0.0 + + nifi-fn-nar + nar + + + + org.apache.nifi + nifi-fn-core + 1.10.0-SNAPSHOT + + + + \ No newline at end of file diff --git a/nifi-fn/nifi-fn-nar/src/main/resources/META-INF/LICENSE b/nifi-fn/nifi-fn-nar/src/main/resources/META-INF/LICENSE new file mode 100644 index 0000000000..27349539cf --- /dev/null +++ b/nifi-fn/nifi-fn-nar/src/main/resources/META-INF/LICENSE @@ -0,0 +1,313 @@ + + 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 'Hamcrest' which is available +under a BSD license. More details found here: http://hamcrest.org. + + Copyright (c) 2000-2006, www.hamcrest.org + All rights reserved. + + Redistribution and use in source and binary forms, with or without + modification, are permitted provided that the following conditions are met: + + Redistributions of source code must retain the above copyright notice, this list of + conditions and the following disclaimer. Redistributions in binary form must reproduce + the above copyright notice, this list of conditions and the following disclaimer in + the documentation and/or other materials provided with the distribution. + + Neither the name of Hamcrest nor the names of its contributors may be used to endorse + or promote products derived from this software without specific prior written + permission. + + THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND ANY + EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES + OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT + SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, + INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED + TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR + BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN + CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY + WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH + DAMAGE. + +The binary distribution of this product bundles 'Antlr 3' which is available +under a "3-clause BSD" license. For details see http://www.antlr3.org/license.html + + Copyright (c) 2010 Terence Parr + All rights reserved. + + Redistribution and use in source and binary forms, with or without + modification, are permitted provided that the following conditions are met: + + Redistributions of source code must retain the above copyright notice, this + list of conditions and the following disclaimer. + Redistributions in binary form must reproduce the above copyright notice, + this list of conditions and the following disclaimer in the documentation + and/or other materials provided with the distribution. + Neither the name of the author nor the names of its contributors may be used + to endorse or promote products derived from this software without specific + prior written permission. + THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" + AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE + IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE + ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE + LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR + CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF + SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS + INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN + CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) + ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF + THE POSSIBILITY OF SUCH DAMAGE. + + +The binary distribution of this product bundles 'Bouncy Castle JDK 1.5' +under an MIT style license. + + Copyright (c) 2000 - 2015 The Legion of the Bouncy Castle Inc. (http://www.bouncycastle.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. + + +This product bundles 'jBCrypt' which is available under an MIT license. +For details see https://github.com/svenkubiak/jBCrypt/blob/0.4.1/LICENSE + + Copyright (c) 2006 Damien Miller + + Permission to use, copy, modify, and distribute this software for any + purpose with or without fee is hereby granted, provided that the above + copyright notice and this permission notice appear in all copies. + + THE SOFTWARE IS PROVIDED "AS IS" AND THE AUTHOR DISCLAIMS ALL WARRANTIES + WITH REGARD TO THIS SOFTWARE INCLUDING ALL IMPLIED WARRANTIES OF + MERCHANTABILITY AND FITNESS. IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR + ANY SPECIAL, DIRECT, INDIRECT, OR CONSEQUENTIAL DAMAGES OR ANY DAMAGES + WHATSOEVER RESULTING FROM LOSS OF USE, DATA OR PROFITS, WHETHER IN AN + ACTION OF CONTRACT, NEGLIGENCE OR OTHER TORTIOUS ACTION, ARISING OUT OF + OR IN CONNECTION WITH THE USE OR PERFORMANCE OF THIS SOFTWARE. + + + + diff --git a/nifi-fn/nifi-fn-nar/src/main/resources/META-INF/NOTICE b/nifi-fn/nifi-fn-nar/src/main/resources/META-INF/NOTICE new file mode 100644 index 0000000000..e0a8fb6f20 --- /dev/null +++ b/nifi-fn/nifi-fn-nar/src/main/resources/META-INF/NOTICE @@ -0,0 +1,144 @@ +nifi-fn-nar +Copyright 2015-2019 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) Jetty + The following NOTICE information applies: + Jetty Web Container + Copyright 1995-2017 Mort Bay Consulting Pty Ltd. + + (ASLv2) Google GSON + The following NOTICE information applies: + Copyright 2008 Google Inc. + + (ASLv2) Apache Commons Text + The following NOTICE information applies: + Apache Commons Text + Copyright 2001-2018 The Apache Software Foundation + + (ASLv2) Jackson JSON processor + The following NOTICE information applies: + # Jackson JSON processor + + Jackson is a high-performance, Free/Open Source JSON processing library. + It was originally written by Tatu Saloranta (tatu.saloranta@iki.fi), and has + been in development since 2007. + It is currently developed by a community of developers, as well as supported + commercially by FasterXML.com. + + ## Licensing + + Jackson core and extension components may licensed under different licenses. + To find the details that apply to this artifact see the accompanying LICENSE file. + For more information, including possible other licensing options, contact + FasterXML.com (http://fasterxml.com). + + ## Credits + + A list of contributors may be found from CREDITS file, which is included + in some artifacts (usually source distributions); but is always available + from the source code management (SCM) system project uses. + + (ASLv2) Apache Commons Codec + The following NOTICE information applies: + Apache Commons Codec + Copyright 2002-2014 The Apache Software Foundation + + src/test/org/apache/commons/codec/language/DoubleMetaphoneTest.java + contains test data from http://aspell.net/test/orig/batch0.tab. + Copyright (C) 2002 Kevin Atkinson (kevina@gnu.org) + + =============================================================================== + + The content of package org.apache.commons.codec.language.bm has been translated + from the original php source code available at http://stevemorse.org/phoneticinfo.htm + with permission from the original authors. + Original source copyright: + Copyright (c) 2008 Alexander Beider & Stephen P. Morse. + + (ASLv2) Apache Commons Lang + The following NOTICE information applies: + Apache Commons Lang + Copyright 2001-2017 The Apache Software Foundation + + This product includes software from the Spring Framework, + under the Apache License 2.0 (see: StringUtils.containsWhitespace()) + + (ASLv2) Quartz + The following NOTICE information applies: + Copyright Declaration: + Copyright © 2003-2016 Software AG, Darmstadt, Germany and/or Software AG USA Inc., Reston, VA, USA, and/or its subsidiaries and/or its affiliates and/or their licensors. + + Trademark and Patent declaration + The name Software AG and all Software AG product names are either trademarks or registered trademarks of Software AG and/or Software AG USA Inc. and/or its subsidiaries and/or its affiliates + and/or their licensors. Other company and product names mentioned herein may be trademarks of their respective owners. + + Detailed information on trademarks and patents owned by Software AG and/or its subsidiaries is located at http://softwareag.com/licenses. + + Third Party declaration + This software may include portions of third-party products. For third-party copyright notices, license terms, additional rights or restrictions, please refer to "License Texts, Copyright + Notices and Disclaimers of Third Party Products". For certain specific third-party license restrictions, please refer to section E of the Legal Notices available under "License Terms and + Conditions for Use of Software AG Products / Copyright and Trademark Notices of Software AG Products". These documents are part of the product documentation, located at + http://softwareag.com/licenses and/or in the root installation directory of the licensed product(s). + + Confidentiality Disclaimer: + Use, reproduction, transfer, publication or disclosure is prohibited except as specifically provided for in your License Agreement with Software AG. + Contact GitHub API Training Shop Blog About + + (ASLv2) Spring Framework + The following NOTICE information applies: + Spring Framework 4.x,5.x.RELEASE + Copyright (c) 2002-2015 Pivotal, Inc. + + (ASLv2) Spring Security + The following NOTICE information applies: + Spring Framework 4.0.3.RELEASE + Copyright (c) 2002-2015 Pivotal, Inc. + + (ASLv2) Ehcache 2.x + The following NOTICE information applies: + Copyright 2003-2010 Terracotta, Inc. + + + + + +************************ +Common Development and Distribution License 1.1 +************************ + +The following binary components are provided under the Common Development and Distribution License 1.1. See project link for details. + + (CDDL 1.1) (GPL2 w/ CPE) Java Servlet API (javax.servlet:javax.servlet-api:jar:3.1.0 - http://servlet-spec.java.net) + (CDDL 1.1) (GPL2 w/ CPE) javax.ws.rs-api (javax.ws.rs:javax.ws.rs-api:jar:2.1 - http://jax-rs-spec.java.net) + (CDDL 1.1) (GPL2 w/ CPE) jersey-client (org.glassfish.jersey.core:jersey-client:jar:2.26 - https://jersey.github.io/) + (CDDL 1.1) (GPL2 w/ CPE) jersey-common (org.glassfish.jersey.core:jersey-common:jar:2.26 - https://jersey.github.io/) + (CDDL 1.1) (GPL2 w/ CPE) jersey-hk2 (org.glassfish.jersey.inject:jersey-hk2:jar:2.26 - https://jersey.github.io/) + (CDDL 1.1) (GPL2 w/ CPE) jersey-media-json-jackson (org.glassfish.jersey.media:jersey-media-json-jackson:jar:2.26 - https://jersey.github.io/) + + +************************ +Eclipse Public License 1.0 +************************ + +The following binary components are provided under the Eclipse Public License 1.0. See project link for details. + + (EPL 1.0)(LGPL 2.1) Logback Classic (ch.qos.logback:logback-classic:jar:1.2.3 - http://logback.qos.ch/) + (EPL 1.0) AspectJ Weaver (org.aspectj:aspectjweaver:jar:1.8.5 - http://www.eclipse.org/aspectj/) + (EPL 1.0) AspectJ Runtime (org.aspectj:aspectjrt:jar:1.8.0 - http://www.eclipse.org/aspectj/) + +***************** +Public Domain +***************** + +The following binary components are provided to the 'Public Domain'. See project link for details. + + (Public Domain) AOP Alliance 1.0 (http://aopalliance.sourceforge.net/) diff --git a/nifi-fn/pom.xml b/nifi-fn/pom.xml index 49c50709ba..2ccb71c9e2 100644 --- a/nifi-fn/pom.xml +++ b/nifi-fn/pom.xml @@ -1,155 +1,37 @@ + - 4.0.0 - org.apache.nifi + + org.apache.nifi + nifi + 1.10.0-SNAPSHOT + + nifi-fn - 1.9.0-SNAPSHOT + pom - - - - org.apache.maven.plugins - maven-compiler-plugin - - 1.8 - 1.8 - - - - org.apache.maven.plugins - maven-dependency-plugin - - - copy-dependencies - prepare-package - - copy-dependencies - - - ${project.build.directory}/lib - false - true - true - - - - - - org.apache.maven.plugins - maven-jar-plugin - - - - true - org.apache.nifi.fn.runtimes.Program - - - - - - com.spotify - dockerfile-maven-plugin - 1.4.8 - - - default - - build - - - - - nifi-fn - ${project.version} - - ${project.build.finalName}.jar - - - - - + + nifi-fn-core + nifi-fn-bootstrap + nifi-fn-assembly + nifi-fn-nar + - - - org.apache.nifi - nifi-framework-core - 1.9.0-SNAPSHOT - - - org.apache.nifi - nifi-framework-nar-loading-utils - 1.9.0-SNAPSHOT - - - org.apache.nifi - nifi-framework-api - 1.9.0-SNAPSHOT - - - org.apache.nifi - nifi-framework-core-api - 1.9.0-SNAPSHOT - - - org.apache.nifi - nifi-web-api - 1.9.0-SNAPSHOT - war - test - - - org.apache.nifi - nifi-nar-utils - 1.9.0-SNAPSHOT - - - org.apache.nifi - nifi-standard-processors - 1.9.0-SNAPSHOT - test - - - org.apache.nifi - nifi-api - 1.9.0-SNAPSHOT - - - org.apache.nifi - nifi-expression-language - 1.9.0-SNAPSHOT - - - org.apache.nifi - nifi-data-provenance-utils - 1.9.0-SNAPSHOT - - - junit - junit - 4.12 - test - - - com.google.code.gson - gson - 2.7 - - - \ No newline at end of file + diff --git a/nifi-fn/src/main/java/org/apache/nifi/fn/core/FnFlow.java b/nifi-fn/src/main/java/org/apache/nifi/fn/core/FnFlow.java deleted file mode 100644 index 25caf6481b..0000000000 --- a/nifi-fn/src/main/java/org/apache/nifi/fn/core/FnFlow.java +++ /dev/null @@ -1,285 +0,0 @@ -/* - * 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.fn.core; - -import com.google.gson.JsonArray; -import com.google.gson.JsonElement; -import com.google.gson.JsonObject; -import org.apache.nifi.bundle.Bundle; -import org.apache.nifi.components.PropertyDescriptor; -import org.apache.nifi.components.ValidationContext; -import org.apache.nifi.components.ValidationResult; -import org.apache.nifi.controller.exception.ProcessorInstantiationException; -import org.apache.nifi.nar.*; -import org.apache.nifi.processor.*; -import org.apache.nifi.processor.exception.ProcessException; -import org.apache.nifi.registry.VariableDescriptor; -import org.apache.nifi.registry.VariableRegistry; -import org.apache.nifi.registry.client.NiFiRegistryException; -import org.apache.nifi.registry.flow.*; -import org.apache.nifi.reporting.InitializationException; -import org.apache.nifi.util.NiFiProperties; - -import java.io.File; -import java.io.IOException; -import java.lang.reflect.InvocationTargetException; -import java.util.*; -import java.util.stream.Collectors; - -public class FnFlow { - - public static final String REGISTRY = "nifi_registry"; - public static final String BUCKETID = "nifi_bucket"; - public static final String FLOWID = "nifi_flow"; - public static final String FLOWVERSION = "nifi_flowversion"; - public static final String MATERIALIZECONTENT = "nifi_materializecontent"; - public static final String FAILUREPORTS = "nifi_failureports"; - public static final String FLOWFILES = "nifi_flowfiles"; - public static final String CONTENT = "nifi_content"; - public static final List reservedFields = Arrays.asList(REGISTRY,BUCKETID,FLOWID,FLOWVERSION,FAILUREPORTS,MATERIALIZECONTENT,FLOWFILES); - - private List roots; - private volatile boolean stopRequested = false; - private FnProcessorWrapper inputProcessor = null; - - public FnFlow(String registryUrl, String bucketID, String flowID, int versionID, VariableRegistry variableRegistry, List failureOutputPorts, boolean materializeContent) throws IOException, NiFiRegistryException, IllegalAccessException, ProcessorInstantiationException, InvocationTargetException, InitializationException { - this( - new RegistryUtil(registryUrl).getFlowByID(bucketID, flowID, versionID), - variableRegistry, - failureOutputPorts, - materializeContent - ); - } - public FnFlow(String registryUrl, String bucketID, String flowID, VariableRegistry variableRegistry, List failureOutputPorts, boolean materializeContent) throws IOException, NiFiRegistryException, IllegalAccessException, ProcessorInstantiationException, InvocationTargetException, InitializationException { - this( - new RegistryUtil(registryUrl).getFlowByID(bucketID, flowID), - variableRegistry, - failureOutputPorts, - materializeContent - ); - } - public FnFlow(VersionedFlowSnapshot flowSnapshot, VariableRegistry variableRegistry, List failureOutputPorts, boolean materializeContent) throws IllegalAccessException, ProcessorInstantiationException, InvocationTargetException, InitializationException { - - VersionedProcessGroup contents = flowSnapshot.getFlowContents(); - Set processors = contents.getProcessors(); - Set connections = contents.getConnections(); - Set inputPorts = contents.getInputPorts(); - Set funnels = contents.getFunnels(); - - if(inputPorts.size() > 1) - throw new IllegalArgumentException("Only one input port per flow is allowed"); - - FnControllerServiceLookup serviceLookup = new FnControllerServiceLookup(); - - Set controllerServices = contents.getControllerServices(); - for(VersionedControllerService service : controllerServices){ - serviceLookup.addControllerService(service); - } - - Map pwMap = new HashMap<>(); - for(VersionedConnection connection : connections) { - boolean isInputPortConnection = false; - FnProcessorWrapper sourcePw = null; - - ConnectableComponent source = connection.getSource(); - switch (source.getType()){ - case PROCESSOR: - if(pwMap.containsKey(source.getId())) { - sourcePw = pwMap.get(source.getId()); - } else { - Optional processor = processors.stream().filter(p -> source.getId().equals(p.getIdentifier())).findFirst(); - if (processor.isPresent()) { - sourcePw = new FnProcessorWrapper(processor.get(), null, serviceLookup, variableRegistry, materializeContent); - pwMap.put(source.getId(), sourcePw); - } else { - throw new IllegalArgumentException("Unknown input processor.. "+source.getId()); - } - } - break; - case REMOTE_INPUT_PORT: - isInputPortConnection = true; - break; - case REMOTE_OUTPUT_PORT: - throw new IllegalArgumentException("Unsupported source type: "+source.getType()); - case INPUT_PORT: - isInputPortConnection = true; - break; - case OUTPUT_PORT: - throw new IllegalArgumentException("Unsupported source type: "+source.getType()); - case FUNNEL: - throw new IllegalArgumentException("Unsupported source type: "+source.getType()); - } - - ConnectableComponent destination = connection.getDestination(); - - FnProcessorWrapper destinationPw; - switch (destination.getType()) { - - case PROCESSOR: - if (pwMap.containsKey(destination.getId())) { - destinationPw = pwMap.get(destination.getId()); - } else { - Optional processor = processors.stream().filter(p -> destination.getId().equals(p.getIdentifier())).findFirst(); - if (!processor.isPresent()) - return; - - destinationPw = new FnProcessorWrapper(processor.get(), sourcePw, serviceLookup, variableRegistry, materializeContent); - pwMap.put(destination.getId(), destinationPw); - } - destinationPw.incomingConnections.add(connection.getIdentifier()); - - if(isInputPortConnection){ - inputProcessor = destinationPw; - } else { - destinationPw.parents.add(sourcePw); - //Link source and destination - for (String relationship : connection.getSelectedRelationships()) { - sourcePw.addChild(destinationPw, new Relationship.Builder().name(relationship).build()); - } - } - break; - case INPUT_PORT: - throw new IllegalArgumentException("Unsupported destination type: "+destination.getType()); - case REMOTE_INPUT_PORT: - throw new IllegalArgumentException("Unsupported destination type: "+destination.getType()); - case REMOTE_OUTPUT_PORT: - case OUTPUT_PORT: - if(isInputPortConnection) - throw new IllegalArgumentException("Input ports can not be mapped directly to output ports..."); - - //Link source and destination - for (String relationship : connection.getSelectedRelationships()) { - sourcePw.addOutputPort( - new Relationship.Builder().name(relationship).build(), - failureOutputPorts.contains(destination.getId()) - ); - } - break; - case FUNNEL: - throw new IllegalArgumentException("Unsupported destination type: "+destination.getType()); - } - } - roots = pwMap.entrySet() - .stream() - .filter(e->e.getValue().parents.isEmpty()) - .map(e->e.getValue()) - .collect(Collectors.toList()); - } - public FnFlow(FnProcessorWrapper root){ - this(Collections.singletonList(root)); - } - public FnFlow(List roots){ - this.roots = roots; - } - - public boolean run(Queue output){ - while(!this.stopRequested){ - for (FnProcessorWrapper pw : roots){ - boolean successful = pw.runRecursive(output); - if(!successful) - return false; - } - } - return true; - } - public boolean runOnce(Queue output){ - for (FnProcessorWrapper pw : roots){ - boolean successful = pw.runRecursive(output); - if(!successful) - return false; - } - return true; - } - public void shutdown(){ - this.stopRequested = true; - this.roots.forEach(r->r.shutdown()); - } - - public static FnFlow createAndEnqueueFromJSON(JsonObject args) throws IllegalAccessException, InvocationTargetException, InitializationException, IOException, ProcessorInstantiationException, NiFiRegistryException { - if(args == null) - throw new IllegalArgumentException("Flow arguments can not be null"); - - System.out.println("Running flow from json: "+args.toString()); - - if(!args.has(REGISTRY) || !args.has(BUCKETID) || !args.has(FLOWID)) - throw new IllegalArgumentException("The following parameters must be provided: "+REGISTRY+", "+BUCKETID+", "+FLOWID); - - String registryurl = args.getAsJsonPrimitive(REGISTRY).getAsString(); - String bucketID = args.getAsJsonPrimitive(BUCKETID).getAsString(); - String flowID = args.getAsJsonPrimitive(FLOWID).getAsString(); - - int flowVersion = -1; - if(args.has(FLOWVERSION)) - flowVersion = args.getAsJsonPrimitive(FLOWVERSION).getAsInt(); - - boolean materializeContent = true; - if(args.has(MATERIALIZECONTENT)) - materializeContent = args.getAsJsonPrimitive(MATERIALIZECONTENT).getAsBoolean(); - - List failurePorts = new ArrayList<>(); - if(args.has(FAILUREPORTS)) - args.getAsJsonArray(FAILUREPORTS).forEach(port-> - failurePorts.add(port.getAsString()) - ); - - Map inputVariables = new HashMap<>(); - - args.entrySet().forEach(entry ->{ - if(!reservedFields.contains(entry.getKey())) - inputVariables.put(new VariableDescriptor(entry.getKey()), entry.getValue().getAsString()); - }); - - FnFlow flow = new FnFlow(registryurl,bucketID,flowID,flowVersion,()->inputVariables, failurePorts, materializeContent); - flow.enqueueFromJSON(args); - return flow; - } - - public void enqueueFlowFile(byte[] content, Map attributes){ - - if(inputProcessor == null) - throw new IllegalArgumentException("Flow does not have an input port..."); - - //enqueue data - Queue input = new LinkedList<>(); - input.add(new FnFlowFile(content,attributes,inputProcessor.materializeContent)); - - inputProcessor.enqueueAll(input); - } - public void enqueueFromJSON(JsonObject json){ - - if(inputProcessor == null) - throw new IllegalArgumentException("Flow does not have an input port..."); - - Queue input = new LinkedList<>(); - JsonArray flowFiles = json.getAsJsonArray(FLOWFILES); - flowFiles.forEach(f->{ - JsonObject file = f.getAsJsonObject(); - - String content = file.getAsJsonPrimitive(CONTENT).getAsString(); - - Map attributes = new HashMap<>(); - file.entrySet().forEach(entry ->{ - if(!CONTENT.equals(entry.getKey())) - attributes.put(entry.getKey(), entry.getValue().getAsString()); - }); - input.add(new FnFlowFile(content,attributes,inputProcessor.materializeContent)); - }); - - //enqueue data - inputProcessor.enqueueAll(input); - } -} diff --git a/nifi-fn/src/main/java/org/apache/nifi/fn/core/FnProcessorWrapper.java b/nifi-fn/src/main/java/org/apache/nifi/fn/core/FnProcessorWrapper.java deleted file mode 100644 index 470bd6ceb8..0000000000 --- a/nifi-fn/src/main/java/org/apache/nifi/fn/core/FnProcessorWrapper.java +++ /dev/null @@ -1,316 +0,0 @@ -/* - * 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.fn.core; - - -import org.apache.nifi.annotation.lifecycle.*; -import org.apache.nifi.components.PropertyDescriptor; -import org.apache.nifi.components.ValidationResult; -import org.apache.nifi.controller.exception.ProcessorInstantiationException; -import org.apache.nifi.logging.ComponentLog; -import org.apache.nifi.processor.Processor; -import org.apache.nifi.processor.Relationship; -import org.apache.nifi.provenance.ProvenanceEventRecord; -import org.apache.nifi.registry.VariableRegistry; -import org.apache.nifi.registry.flow.VersionedProcessor; - -import java.lang.reflect.InvocationTargetException; -import java.util.*; -import java.util.concurrent.CopyOnWriteArraySet; -import java.util.concurrent.atomic.AtomicBoolean; -import java.util.stream.Collectors; - - -public class FnProcessorWrapper { - - private final long runSchedule = 100; - - public List parents; - public List incomingConnections; - public final Map> children; - private final Set autoTermination; - private final Set successOutputPorts; - private final Set failureOutputPorts; - - - public boolean materializeContent; - private final Processor processor; - private final FnProcessContext context; - private final Queue inputQueue; - private final VariableRegistry variableRegistry; - - private final Collection provenanceEvents; - - private final Set createdSessions; - private final ComponentLog logger; - - private final FnControllerServiceLookup lookup; - - private volatile boolean stopRequested = false; - private volatile boolean isStopped = true; - private volatile boolean initialized = false; - - FnProcessorWrapper(final VersionedProcessor processor, final FnProcessorWrapper parent, FnControllerServiceLookup lookup, VariableRegistry registry, boolean materializeContent) throws InvocationTargetException, IllegalAccessException, ProcessorInstantiationException { - this(ReflectionUtils.createProcessor(processor),parent, lookup, registry,materializeContent); - for(String relationship : processor.getAutoTerminatedRelationships()) { - this.addAutoTermination(new Relationship.Builder().name(relationship).build()); - } - processor.getProperties().forEach((key, value) -> this.setProperty(key,value)); - } - - FnProcessorWrapper(final Processor processor, final FnProcessorWrapper parent, FnControllerServiceLookup lookup, VariableRegistry registry, boolean materializeContent) throws InvocationTargetException, IllegalAccessException { - - this.processor = processor; - this.parents = new ArrayList<>(); - if(parent != null) - this.parents.add(parent); - this.lookup = lookup; - this.materializeContent = materializeContent; - - this.incomingConnections = new ArrayList<>(); - this.children = new HashMap<>(); - this.autoTermination = new HashSet<>(); - this.successOutputPorts = new HashSet<>(); - this.failureOutputPorts = new HashSet<>(); - - this.provenanceEvents = new ArrayList<>(); - this.createdSessions = new CopyOnWriteArraySet<>(); - this.inputQueue = new LinkedList<>(); - this.variableRegistry = registry; - this.context = new FnProcessContext(processor, lookup, processor.getIdentifier(), new FnStateManager(), variableRegistry); - this.context.setMaxConcurrentTasks(1); - - final FnProcessorInitializationContext initContext = new FnProcessorInitializationContext(processor, context); - processor.initialize(initContext); - logger = initContext.getLogger(); - - ReflectionUtils.invokeMethodsWithAnnotation(OnAdded.class, processor); - - ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnConfigurationRestored.class, processor); - } - - - public Processor getProcessor(){return this.processor;} - - private void initialize(){ - - //Validate context - Collection validationResult = context.validate(); - if(validationResult.stream().anyMatch(a->!a.isValid()) || !this.validate()) { - throw new IllegalArgumentException( - "context is not valid: "+ - String.join("\n",validationResult.stream().map(r->r.toString()).collect(Collectors.toList()))); - } - try { - ReflectionUtils.invokeMethodsWithAnnotation(OnScheduled.class, processor, context); - } catch (IllegalAccessException | InvocationTargetException e) { - logger.error("Exception: ",e); - } - initialized = true; - } - public boolean runRecursive(Queue output) { - if(!initialized) - initialize(); - - AtomicBoolean processingSuccess = new AtomicBoolean(true); - Set outputRelationships = new HashSet<>(children.keySet()); - outputRelationships.addAll(successOutputPorts); - outputRelationships.addAll(failureOutputPorts); - do { - this.isStopped = false; - AtomicBoolean nextStepCalled = new AtomicBoolean(false); - try { - logger.info("Running "+this.processor.getClass().getSimpleName()+".onTrigger with "+inputQueue.size()+" records"); - processor.onTrigger(context, () -> { - final FnProcessSession session = new FnProcessSession( - inputQueue, - provenanceEvents, - processor, - outputRelationships, - materializeContent, - () -> { - if(!nextStepCalled.get()) { - nextStepCalled.set(true); - boolean successfulRun = runChildren(output); - processingSuccess.set(successfulRun); - } - }); - createdSessions.add(session); - return session; - }); - if(!nextStepCalled.get()) { - nextStepCalled.set(true); - boolean successfulRun = runChildren(output); - processingSuccess.set(successfulRun); - } - provenanceEvents.clear(); - Thread.sleep(runSchedule); - } catch (final Exception t) { - logger.error("Exception in runRecursive "+this.processor.getIdentifier(),t); - return false; - } - } while(!stopRequested && !inputQueue.isEmpty() && processingSuccess.get()); - this.isStopped = true; - return processingSuccess.get(); - } - - private boolean runChildren(Queue output) { - Queue penalizedFlowFiles = this.getPenalizedFlowFiles(); - if(penalizedFlowFiles.size() > 0){ - output.addAll(penalizedFlowFiles); - return false; - } - - for(Relationship r : this.getProcessor().getRelationships()) { - if(this.autoTermination.contains(r)) - continue; - - Queue files = this.getAndRemoveFlowFilesForRelationship(r); - if(files.size() == 0) - continue; - - if(this.failureOutputPorts.contains(r)) { - output.addAll(files); - return false; - } - if(this.successOutputPorts.contains(r)) - output.addAll(files); - - if(children.containsKey(r)) { - for (FnProcessorWrapper child : children.get(r)) { - child.enqueueAll(files); - boolean successfulRun = child.runRecursive(output); - if (!successfulRun) - return false; - } - } - } - return true; - } - public void shutdown(){ - this.stopRequested = true; - for(Relationship r : this.getProcessor().getRelationships()) { - if(this.autoTermination.contains(r)) - continue; - - if(!children.containsKey(r)) - throw new IllegalArgumentException("No child for relationship: "+r.getName()); - - children.get(r).forEach(FnProcessorWrapper::shutdown); - } - - while(!this.isStopped){ - try { - Thread.sleep(10); - } catch (InterruptedException e) { - e.printStackTrace(); - break; - } - } - try { - ReflectionUtils.invokeMethodsWithAnnotation(OnUnscheduled.class, processor, context); - - ReflectionUtils.invokeMethodsWithAnnotation(OnStopped.class, processor, context); - - ReflectionUtils.invokeMethodsWithAnnotation(OnShutdown.class, processor); - } catch (IllegalAccessException | InvocationTargetException e) { - logger.error("Failure on shutdown: ", e); - } - logger.info(this.processor.getClass().getSimpleName()+" shutdown"); - } - public boolean validate(){ - if(!context.isValid()) - return false; - - for(Relationship r : this.getProcessor().getRelationships()) { - boolean hasChildren = this.children.containsKey(r); - boolean hasAutoterminate = this.autoTermination.contains(r); - boolean hasFailureOutputPort = this.failureOutputPorts.contains(r); - boolean hasSuccessOutputPort = this.successOutputPorts.contains(r); - if (!(hasChildren || hasAutoterminate || hasFailureOutputPort || hasSuccessOutputPort)) { - logger.error("Processor: " + this.toString() + ", Relationship: " + r.getName() + ", needs either auto terminate, child processors, or an output port"); - return false; - } - } - for( Map.Entry> child : this.children.entrySet()){ - for(FnProcessorWrapper n : child.getValue()){ - if(!n.validate()) - return false; - } - } - return true; - } - - - public void enqueueAll(Queue list){ - inputQueue.addAll(list); - } - public Queue getAndRemoveFlowFilesForRelationship(final Relationship relationship) { - - List sortedList = createdSessions.stream() - .flatMap(s-> s.getAndRemoveFlowFilesForRelationship(relationship).stream()) - .sorted(Comparator.comparing(f -> f.getCreationTime())) - .collect(Collectors.toList()); - - return new LinkedList<>(sortedList); - } - public Queue getPenalizedFlowFiles(){ - List sortedList = createdSessions.stream() - .flatMap(s-> s.getPenalizedFlowFiles().stream()) - .sorted(Comparator.comparing(f -> f.getCreationTime())) - .collect(Collectors.toList()); - return new LinkedList<>(sortedList); - - } - - public ValidationResult setProperty(final PropertyDescriptor property, final String propertyValue) { - return context.setProperty(property,propertyValue); - } - public ValidationResult setProperty(final String propertyName, final String propertyValue) { - return context.setProperty(propertyName, propertyValue); - } - public void addOutputPort(Relationship relationship, boolean isFailurePort){ - if(isFailurePort) - this.failureOutputPorts.add(relationship); - else - this.successOutputPorts.add(relationship); - } - - public FnProcessorWrapper addChild(Processor p, Relationship relationship) throws InvocationTargetException, IllegalAccessException { - ArrayList list = children.computeIfAbsent(relationship, r -> new ArrayList<>()); - FnProcessorWrapper child = new FnProcessorWrapper(p,this, lookup, variableRegistry, materializeContent); - list.add(child); - - context.addConnection(relationship); - return child; - } - - public FnProcessorWrapper addChild(FnProcessorWrapper child, Relationship relationship) { - ArrayList list = children.computeIfAbsent(relationship, r -> new ArrayList<>()); - list.add(child); - - context.addConnection(relationship); - return child; - } - public void addAutoTermination(Relationship relationship){ - this.autoTermination.add(relationship); - - context.addConnection(relationship); - } - -} - diff --git a/nifi-fn/src/main/java/org/apache/nifi/fn/core/RegistryUtil.java b/nifi-fn/src/main/java/org/apache/nifi/fn/core/RegistryUtil.java deleted file mode 100644 index efbe2e83d4..0000000000 --- a/nifi-fn/src/main/java/org/apache/nifi/fn/core/RegistryUtil.java +++ /dev/null @@ -1,90 +0,0 @@ -/* - * 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.fn.core; - -import org.apache.nifi.authorization.user.NiFiUserUtils; -import org.apache.nifi.registry.bucket.Bucket; -import org.apache.nifi.registry.client.NiFiRegistryException; -import org.apache.nifi.registry.flow.*; -import org.apache.nifi.util.NiFiProperties; - -import javax.net.ssl.SSLContext; -import java.io.IOException; -import java.security.NoSuchAlgorithmException; -import java.util.Map; -import java.util.Optional; -import java.util.Set; - -public class RegistryUtil { - private RestBasedFlowRegistry registry; - public RegistryUtil(String registryUrl){ - try { - registry = new RestBasedFlowRegistry(new StandardFlowRegistryClient(),"id",registryUrl, SSLContext.getDefault(),"name"); - } catch (NoSuchAlgorithmException e) { - e.printStackTrace(); - } - } - public VersionedFlowSnapshot getFlowByName(String bucketName, String flowName) throws IOException, NiFiRegistryException { - return getFlowByName(bucketName, flowName,-1); - } - public VersionedFlowSnapshot getFlowByName(String bucketName, String flowName, int versionID) throws IOException, NiFiRegistryException { - - //Get bucket by name - Set buckets = this.getBuckets(); - Optional bucketOptional = buckets.stream().filter(b->b.getName().equals(bucketName)).findFirst(); - if(!bucketOptional.isPresent()) - throw new IllegalArgumentException("Bucket not found"); - String bucketID = bucketOptional.get().getIdentifier(); - - - //Get flow by name - Set flows = this.getFlows(bucketID); - Optional flowOptional = flows.stream().filter(b->b.getName().equals(flowName)).findFirst(); - if(!flowOptional.isPresent()) - throw new IllegalArgumentException("Flow not found"); - String flowID = flowOptional.get().getIdentifier(); - - return getFlowByID(bucketID,flowID, versionID); - } - public VersionedFlowSnapshot getFlowByID(String bucketID, String flowID) throws IOException, NiFiRegistryException { - return getFlowByID(bucketID, flowID,-1); - } - public VersionedFlowSnapshot getFlowByID(String bucketID, String flowID, int versionID) throws IOException, NiFiRegistryException { - if(versionID == -1) - versionID = this.getLatestVersion(bucketID, flowID); - return registry.getFlowContents(bucketID, flowID, versionID,true); - } - - public Map getVariables(String bucketID, String flowID) throws IOException, NiFiRegistryException { - VersionedFlowSnapshot flow = this.getFlowByID(bucketID,flowID); - return flow.getFlowContents().getVariables(); - } - public Map getVariables(String bucketID, String flowID, int versionID) throws IOException, NiFiRegistryException { - VersionedFlowSnapshot flow = this.getFlowByID(bucketID,flowID,versionID); - return flow.getFlowContents().getVariables(); - } - - public Set getBuckets() throws IOException, NiFiRegistryException { - return registry.getBuckets(NiFiUserUtils.getNiFiUser()); - } - public Set getFlows(String bucketID) throws IOException, NiFiRegistryException { - return registry.getFlows(bucketID,NiFiUserUtils.getNiFiUser()); - } - public int getLatestVersion(String bucketID, String flowID) throws IOException, NiFiRegistryException { - return registry.getLatestVersion(bucketID, flowID, NiFiUserUtils.getNiFiUser()); - } -} diff --git a/nifi-fn/src/main/java/org/apache/nifi/fn/runtimes/Program.java b/nifi-fn/src/main/java/org/apache/nifi/fn/runtimes/Program.java deleted file mode 100644 index bf8a9152a6..0000000000 --- a/nifi-fn/src/main/java/org/apache/nifi/fn/runtimes/Program.java +++ /dev/null @@ -1,185 +0,0 @@ -/* - * 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.fn.runtimes; - -import com.google.gson.JsonObject; -import com.google.gson.JsonParser; -import org.apache.nifi.fn.core.FnFlow; -import org.apache.nifi.fn.core.FnFlowFile; -import org.apache.nifi.fn.runtimes.OpenWhisk.NiFiFnOpenWhiskAction; -import org.apache.nifi.fn.runtimes.YARN.YARNServiceUtil; -import org.apache.nifi.controller.exception.ProcessorInstantiationException; -import org.apache.nifi.registry.VariableDescriptor; -import org.apache.nifi.registry.client.NiFiRegistryException; -import org.apache.nifi.reporting.InitializationException; - -import java.io.IOException; -import java.lang.reflect.InvocationTargetException; -import java.nio.file.Files; -import java.nio.file.Paths; -import java.util.*; - -public class Program { - - public static final String RUN_FROM_REGISTRY = "RunFromRegistry"; - public static final String RUN_YARN_SERVICE_FROM_REGISTRY = "RunYARNServiceFromRegistry"; - public static final String RUN_OPENWHISK_ACTION_SERVER = "RunOpenwhiskActionServer"; - - public static void main(String[] args) throws InvocationTargetException, IllegalAccessException, ProcessorInstantiationException, NiFiRegistryException, IOException, InterruptedException, InitializationException { - if(args.length == 0) { - printUsage(); - System.exit(1); - } else if(args[0].equals(RUN_FROM_REGISTRY) && (args[1].equals("Once") || args[1].equals("Continuous")) && args.length >= 4) { - - boolean once = args[1].equals("Once"); - FnFlow flow; - if(args[2].equals("--file")){ - String json = new String(Files.readAllBytes(Paths.get(args[3]))); - JsonObject config = new JsonParser().parse(json).getAsJsonObject(); - flow = FnFlow.createAndEnqueueFromJSON(config); - } else if (args[2].equals("--json")){ - JsonObject config = new JsonParser().parse(args[3]).getAsJsonObject(); - flow = FnFlow.createAndEnqueueFromJSON(config); - } else if(args.length >= 5) { - //Initialize flow - String registryUrl = args[2]; - String bucketID = args[3]; - String flowID = args[4]; - Map inputVariables = new HashMap<>(); - - String[] variables = args[5].split(";"); - for (String v : variables) { - String[] tokens = v.split("-"); - inputVariables.put(new VariableDescriptor(tokens[0]), tokens[1]); - } - - String[] failureOutputPorts = args.length >= 7 ? args[6].split(";") : new String[]{}; - flow = new FnFlow(registryUrl, bucketID, flowID, () -> inputVariables, Arrays.asList(failureOutputPorts), true); - - //Enqueue all provided flow files - if (7 < args.length) { - int i = 7; - while (i < args.length) { - Map attributes = new HashMap<>(); - byte[] content = {}; - - String[] attributesArr = args[i].split(";"); - for (String v : attributesArr) { - String[] tokens = v.split("-"); - if (tokens[0].equals(FnFlow.CONTENT)) - content = tokens[1].getBytes(); - else - attributes.put(tokens[0], tokens[1]); - } - flow.enqueueFlowFile(content, attributes); - i++; - } - } - } else { - System.out.println("Invalid input: "+String.join(",",args)); - printUsage(); - System.exit(1); - return; - } - - //Run Flow - Queue outputFlowFiles = new LinkedList<>(); - //run flow once or forever - boolean successful; - if (once) - successful = flow.runOnce(outputFlowFiles); - else - successful = flow.run(outputFlowFiles);//Run forever - - if(!successful) { - System.out.println("Flow Failed"); - outputFlowFiles.forEach(f->System.out.println(f.toStringFull())); - System.exit(1); - } else { - System.out.println("Flow Succeeded"); - outputFlowFiles.forEach(f->System.out.println(f.toStringFull())); - } - - }else if(args[0].equals(RUN_YARN_SERVICE_FROM_REGISTRY) && args.length >= 7){ - String YARNUrl = args[1]; - String imageName = args[2]; - String serviceName = args[3]; - int numberOfContainers = Integer.parseInt(args[4]); - List launchCommand = Arrays.asList(RUN_FROM_REGISTRY,"Continuous"); - - if(args[5].equals("--file")){ - launchCommand.add("--json"); - launchCommand.add(new String(Files.readAllBytes(Paths.get(args[6])))); - } else if (args[5].equals("--json")){ - launchCommand.add("--json"); - launchCommand.add(args[6]); - } if(args.length >= 9) { - for(int i = 5; i < args.length; i++){ - launchCommand.add(args[i]); - } - } else { - System.out.println("Invalid input: "+String.join(",",args)); - printUsage(); - System.exit(1); - } - - StringBuilder message = new StringBuilder(); - YARNServiceUtil yarnServiceUtil = new YARNServiceUtil(YARNUrl, imageName); - yarnServiceUtil.launchYARNService(serviceName, numberOfContainers, launchCommand.toArray(new String[0]), message); - System.out.println(message); - } else if (args[0].equals(RUN_OPENWHISK_ACTION_SERVER) && args.length == 2){ - NiFiFnOpenWhiskAction action = new NiFiFnOpenWhiskAction(Integer.parseInt(args[1])); - action.start(); - }else{ - System.out.println("Invalid input: "+String.join(",",args)); - printUsage(); - System.exit(1); - } - } - - private static void printUsage(){ - System.out.println("Usage:"); - System.out.println(" 1) "+RUN_FROM_REGISTRY+" [Once|Continuous] [] []"); - System.out.println(" "+RUN_FROM_REGISTRY+" [Once|Continuous] --json "); - System.out.println(" "+RUN_FROM_REGISTRY+" [Once|Continuous] --file "); - System.out.println(); - System.out.println(" 2) "+RUN_YARN_SERVICE_FROM_REGISTRY+" <# of Containers> \\"); - System.out.println(" [] []"); - System.out.println(" "+RUN_YARN_SERVICE_FROM_REGISTRY+" <# of Containers> --json "); - System.out.println(" "+RUN_YARN_SERVICE_FROM_REGISTRY+" <# of Containers> --file "); - System.out.println(); - System.out.println(" 3) "+RUN_OPENWHISK_ACTION_SERVER+" "); - System.out.println(); - System.out.println("Examples:"); - System.out.println(" 1) "+RUN_FROM_REGISTRY+" Once http://172.0.0.1:61080 e53b8a0d-5c85-4fcd-912a-1c549a586c83 6cf8277a-c402-4957-8623-0fa9890dd45d \\"); - System.out.println(" \"DestinationDirectory-/tmp/nififn/output2/\" \"\" \"absolute.path-/tmp/nififn/input/;filename-test.txt\" \"absolute.path-/tmp/nififn/input/;filename-test2.txt\""); - System.out.println(" 2) "+RUN_FROM_REGISTRY+" Once http://172.0.0.1:61080 e53b8a0d-5c85-4fcd-912a-1c549a586c83 6cf8277a-c402-4957-8623-0fa9890dd45d \\"); - System.out.println(" \"DestinationDirectory-/tmp/nififn/output2/\" \"f25c9204-6c95-3aa9-b0a8-c556f5f61849\" \"absolute.path-/tmp/nififn/input/;filename-test.txt\""); - System.out.println(" 3) "+RUN_YARN_SERVICE_FROM_REGISTRY+" http://127.0.0.1:8088 nifi-fn:latest kafka-to-solr 3 --file kafka-to-solr.json"); - System.out.println(" 4) "+RUN_OPENWHISK_ACTION_SERVER+" 8080"); - System.out.println(); - System.out.println("Notes:"); - System.out.println(" 1) will be split on ';' and '-' then injected into the flow using the variable registry interface."); - System.out.println(" 2) will be split on ';'. FlowFiles routed to matching output ports will immediately fail the flow."); - System.out.println(" 3) will be split on ';' and '-' then injected into the flow using the \""+FnFlow.CONTENT+"\" field as the FlowFile content."); - System.out.println(" 4) Multiple arguments can be provided."); - System.out.println(" 5) The configuration file must be in JSON format. "); - System.out.println(" 6) When providing configurations via JSON, the following attributes must be provided: "+ FnFlow.REGISTRY+", "+ FnFlow.BUCKETID+", "+ FnFlow.FLOWID+"."); - System.out.println(" All other attributes will be passed to the flow using the variable registry interface"); - System.out.println(); - } -} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/nar/StandardExtensionDiscoveringManager.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/nar/StandardExtensionDiscoveringManager.java index adadea116a..a24d29c4ed 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/nar/StandardExtensionDiscoveringManager.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/nar/StandardExtensionDiscoveringManager.java @@ -46,6 +46,7 @@ import java.io.IOException; import java.net.URL; import java.net.URLClassLoader; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collections; import java.util.HashMap; import java.util.HashSet; @@ -238,7 +239,7 @@ public class StandardExtensionDiscoveringManager implements ExtensionDiscovering if (!cobundledApis.isEmpty()) { logger.warn(String.format( "Component %s is bundled with its referenced Controller Service APIs %s. The service APIs should not be bundled with component implementations that reference it.", - component.getClass().getName(), StringUtils.join(cobundledApis.stream().map(cls -> cls.getName()).collect(Collectors.toSet()), ", "))); + component.getClass().getName(), StringUtils.join(cobundledApis.stream().map(Class::getName).collect(Collectors.toSet()), ", "))); } // the component is eligible when it does not require instance classloading or when the supporting APIs are bundled in a parent NAR @@ -265,7 +266,7 @@ public class StandardExtensionDiscoveringManager implements ExtensionDiscovering if (!cobundledApis.isEmpty()) { logger.warn(String.format("Controller Service %s is bundled with its supporting APIs %s. The service APIs should not be bundled with the implementations.", - originalExtensionType.getName(), StringUtils.join(cobundledApis.stream().map(cls -> cls.getName()).collect(Collectors.toSet()), ", "))); + originalExtensionType.getName(), StringUtils.join(cobundledApis.stream().map(Class::getName).collect(Collectors.toSet()), ", "))); } // the service is eligible when it does not require instance classloading or when the supporting APIs are bundled in a parent NAR @@ -364,9 +365,7 @@ public class StandardExtensionDiscoveringManager implements ExtensionDiscovering logger.debug("Including ClassLoader resources from {} for component {}", new Object[] {bundle.getBundleDetails(), instanceIdentifier}); final Set instanceUrls = new LinkedHashSet<>(); - for (final URL url : narBundleClassLoader.getURLs()) { - instanceUrls.add(url); - } + instanceUrls.addAll(Arrays.asList(narBundleClassLoader.getURLs())); ClassLoader ancestorClassLoader = narBundleClassLoader.getParent(); @@ -374,7 +373,7 @@ public class StandardExtensionDiscoveringManager implements ExtensionDiscovering final ConfigurableComponent component = getTempComponent(classType, bundle.getBundleDetails().getCoordinate()); final Set reachableApiBundles = findReachableApiBundles(component); - while (ancestorClassLoader != null && ancestorClassLoader instanceof NarClassLoader) { + while (ancestorClassLoader instanceof NarClassLoader) { final Bundle ancestorNarBundle = classLoaderBundleLookup.get(ancestorClassLoader); // stop including ancestor resources when we reach one of the APIs, or when we hit the Jetty NAR @@ -384,9 +383,7 @@ public class StandardExtensionDiscoveringManager implements ExtensionDiscovering } final NarClassLoader ancestorNarClassLoader = (NarClassLoader) ancestorClassLoader; - for (final URL url : ancestorNarClassLoader.getURLs()) { - instanceUrls.add(url); - } + Collections.addAll(instanceUrls, ancestorNarClassLoader.getURLs()); ancestorClassLoader = ancestorNarClassLoader.getParent(); } } @@ -498,7 +495,7 @@ public class StandardExtensionDiscoveringManager implements ExtensionDiscovering throw new IllegalArgumentException("Class cannot be null"); } final Set extensions = definitionMap.get(definition); - return (extensions == null) ? Collections.emptySet() : extensions; + return (extensions == null) ? Collections.emptySet() : extensions; } @Override @@ -527,8 +524,7 @@ public class StandardExtensionDiscoveringManager implements ExtensionDiscovering builder.append("\n\t=== ").append(entry.getKey().getSimpleName()).append(" Type ==="); for (final Class type : entry.getValue()) { - final List bundles = classNameBundleLookup.containsKey(type.getName()) - ? classNameBundleLookup.get(type.getName()) : Collections.emptyList(); + final List bundles = classNameBundleLookup.getOrDefault(type.getName(), Collections.emptyList()); builder.append("\n\t").append(type.getName()); diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/nar/NarClassLoaders.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/nar/NarClassLoaders.java index 02fcaada74..380ade06cb 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/nar/NarClassLoaders.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/nar/NarClassLoaders.java @@ -34,6 +34,7 @@ import java.util.LinkedHashMap; import java.util.LinkedHashSet; import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.Optional; import java.util.Set; @@ -106,9 +107,10 @@ public final class NarClassLoaders { */ public void init(final ClassLoader rootClassloader, final File frameworkWorkingDir, final File extensionsWorkingDir) throws IOException, ClassNotFoundException { - if (frameworkWorkingDir == null || extensionsWorkingDir == null) { + if (extensionsWorkingDir == null) { throw new NullPointerException("cannot have empty arguments"); } + InitContext ic = initContext; if (ic == null) { synchronized (this) { @@ -118,8 +120,10 @@ public final class NarClassLoaders { } } } + boolean matching = initContext.extensionWorkingDir.equals(extensionsWorkingDir) - && initContext.frameworkWorkingDir.equals(frameworkWorkingDir); + && Objects.equals(initContext.frameworkWorkingDir, frameworkWorkingDir); + if (!matching) { throw new IllegalStateException("Cannot reinitialize and extension/framework directories cannot change"); } @@ -138,14 +142,17 @@ public final class NarClassLoaders { final Map> narIdBundleLookup = new HashMap<>(); // make sure the nar directory is there and accessible - FileUtils.ensureDirectoryExistAndCanReadAndWrite(frameworkWorkingDir); - FileUtils.ensureDirectoryExistAndCanReadAndWrite(extensionsWorkingDir); - final List narWorkingDirContents = new ArrayList<>(); - final File[] frameworkWorkingDirContents = frameworkWorkingDir.listFiles(); - if (frameworkWorkingDirContents != null) { - narWorkingDirContents.addAll(Arrays.asList(frameworkWorkingDirContents)); + + if (frameworkWorkingDir != null) { + FileUtils.ensureDirectoryExistAndCanReadAndWrite(frameworkWorkingDir); + final File[] frameworkWorkingDirContents = frameworkWorkingDir.listFiles(); + if (frameworkWorkingDirContents != null) { + narWorkingDirContents.addAll(Arrays.asList(frameworkWorkingDirContents)); + } } + + FileUtils.ensureDirectoryExistAndCanReadAndWrite(extensionsWorkingDir); final File[] extensionsWorkingDirContents = extensionsWorkingDir.listFiles(); if (extensionsWorkingDirContents != null) { narWorkingDirContents.addAll(Arrays.asList(extensionsWorkingDirContents)); diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/nar/NarUnpacker.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/nar/NarUnpacker.java index 8bc163c62c..5830f8803d 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/nar/NarUnpacker.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/nar/NarUnpacker.java @@ -54,7 +54,7 @@ import java.util.jar.Manifest; * */ public final class NarUnpacker { - + public static final String BUNDLED_DEPENDENCIES_DIRECTORY = "NAR-INF/bundled-dependencies"; private static final Logger logger = LoggerFactory.getLogger(NarUnpacker.class); private static String HASH_FILENAME = "nar-md5sum"; private static final FileFilter NAR_FILTER = new FileFilter() { @@ -210,14 +210,14 @@ public final class NarUnpacker { final File unpackedNar = entry.getKey(); final BundleCoordinate bundleCoordinate = entry.getValue(); - final File bundledDependencies = new File(unpackedNar, "NAR-INF/bundled-dependencies"); + final File bundledDependencies = new File(unpackedNar, BUNDLED_DEPENDENCIES_DIRECTORY); unpackBundleDocs(docsDirectory, mapping, bundleCoordinate, bundledDependencies); } } public static void mapExtension(final File unpackedNar, final BundleCoordinate bundleCoordinate, final File docsDirectory, final ExtensionMapping mapping) throws IOException { - final File bundledDependencies = new File(unpackedNar, "NAR-INF/bundled-dependencies"); + final File bundledDependencies = new File(unpackedNar, BUNDLED_DEPENDENCIES_DIRECTORY); unpackBundleDocs(docsDirectory, mapping, bundleCoordinate, bundledDependencies); } @@ -282,7 +282,7 @@ public final class NarUnpacker { JarEntry jarEntry = jarEntries.nextElement(); String name = jarEntry.getName(); if(name.contains("META-INF/bundled-dependencies")){ - name = name.replace("META-INF/bundled-dependencies", "NAR-INF/bundled-dependencies"); + name = name.replace("META-INF/bundled-dependencies", BUNDLED_DEPENDENCIES_DIRECTORY); } File f = new File(workingDirectory, name); if (jarEntry.isDirectory()) { diff --git a/pom.xml b/pom.xml index c8b7b53342..5dabbbca03 100644 --- a/pom.xml +++ b/pom.xml @@ -36,6 +36,7 @@ nifi-external nifi-toolkit nifi-docker + nifi-fn http://nifi.apache.org